From 04ce6ef0ab9fe34ddee7eabe3c592e9f20b02977 Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Wed, 25 Jun 2014 18:03:44 +0200 Subject: [PATCH 001/182] ISSUE 827 fix Adding WordCountPLOJO example. It demonstrates how to use KeySelectors in a word count example. Removing comments and adding file support so that this example is executable without additional data. While providing the functionality of adding input and output files. --- .../java/wordcount/WordCountPLOJO.java | 183 ++++++++++++++++++ 1 file changed, 183 insertions(+) create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java new file mode 100644 index 0000000000000..c8525d5590e32 --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java @@ -0,0 +1,183 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.wordcount; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.functions.FlatMapFunction; +import eu.stratosphere.api.java.functions.KeySelector; +import eu.stratosphere.api.java.functions.ReduceFunction; +import eu.stratosphere.util.Collector; + + + +/** + * Implements a "WordCount" program that computes a simple word occurrence histogram + * over hard coded examples or text files. This example demonstrates how to use KeySelectors, ReduceFunction and FlatMapFunction. + */ +@SuppressWarnings("serial") +public class WordCountPLOJO { + + /** + * Runs the WordCount program. + * + * @param args Input and output file. + */ + public static void main(String[] args) throws Exception { + // Check whether arguments are given and tell user how to use this example with files. + if (args.length < 2) { + System.out.println("You can specify: WordCountPLOJO , in order to work with files."); + } + + // Input and output path [optional] + String inputPath = null; + String outputPath = null; + + // Get the environment as starting point + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // Read the text file from given input path or hard coded + DataSet text = null; + try { + inputPath = args[0]; + env.readTextFile(inputPath); + } + catch(Exception e) { + System.out.println("No input file specified. Using hard coded example."); + text = env.fromElements("To be", "or not to be", "or to be still", "and certainly not to be not at all", "is that the question?"); + } + + // Split up the lines in pairs (2-tuples) containing: (word,1) + DataSet words = text.flatMap(new Tokenizer()); + + // Create KeySelector to be able to group CustomizedWord + CustomizedWordKeySelector keySelector = new CustomizedWordKeySelector(); + + // Instantiate customized reduce function + CustomizedWordReducer reducer = new CustomizedWordReducer(); + + // Group by the tuple field "0" and sum up tuple field "1" + DataSet result = words.groupBy(keySelector).reduce(reducer); + + // Print result + try { + outputPath = args[1]; + // write out the result + result.writeAsText(outputPath); + } + catch(Exception e) { + System.out.println("No output file specified. Printing result to console."); + // Print result to console + result.print(); + } + + // Execute the defined program + env.execute("Word Count"); + } + + /** + * Implements the string tokenizer that splits sentences into words as a user-defined + * FlatMapFunction. The function takes a line (String) and splits it into + * multiple pairs in the form of "(word,1)" CustomizedWord). + */ + public static final class Tokenizer extends FlatMapFunction { + + @Override + public void flatMap(String value, Collector out) { + // Normalize and split the line + String[] tokens = value.toLowerCase().split("\\W+"); + + // Emit the pairs + for (String token : tokens) { + if (token.length() > 0) { + out.collect(new CustomizedWord(token, 1)); + } + } + } + } + + /** + * Customized reducer for CustomizedWord. + */ + public static class CustomizedWordReducer extends ReduceFunction{ + + /** + * This function is applied to all members of a group/ Hence to all words which to match. + * Only the count needs to be adjusted. + */ + @Override + public CustomizedWord reduce(CustomizedWord value1, + CustomizedWord value2) throws Exception { + value2.count += value1.count; + return value2; + } + + } + + /** + * This class is a customized word and count class. It represents a Tuple with two entries (word,count). + * For this example a customized class is used in order to show how to use KeySelectors + */ + public static class CustomizedWord{ + // Word + public String word; + // Count of how often word was found + public int count; + + /** + * Standard constructor. + */ + public CustomizedWord() { + } + + /** + * Constructor to set public members of class. + * @param word The word. + * @param count The number of appearances. + */ + public CustomizedWord(String word, int count) { + + // Set values + this.word = word; + this.count = count; + } + + /** + * Convert to String. For a nice printed result. + */ + @Override + public String toString() { + return "<"+word+","+count+">"; + } + } + + /** + * KeySelector written for CustomizedWord. + * This implementation extracts the KEY out of CustomizedWord. + * + */ + public static class CustomizedWordKeySelector extends KeySelector { + + /** + * Extract the KEY out of CustomizedWord. + */ + @Override + public String getKey(CustomizedWord value) { + // Access the word (String) , which is key. + return value.word; + } + + } +} From a4a539d3ede4cb4739435ae6fe94a2572559aec1 Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Thu, 26 Jun 2014 08:50:04 +0200 Subject: [PATCH 002/182] Renaming and inline instantiation Additionally no exceptions are caught when no arguments are give. Instead the array length is used to determine how many arguments where given. --- ...WordCountPLOJO.java => WordCountPOJO.java} | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/{WordCountPLOJO.java => WordCountPOJO.java} (91%) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPOJO.java similarity index 91% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPOJO.java index c8525d5590e32..e98325b10b063 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPLOJO.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPOJO.java @@ -28,7 +28,7 @@ * over hard coded examples or text files. This example demonstrates how to use KeySelectors, ReduceFunction and FlatMapFunction. */ @SuppressWarnings("serial") -public class WordCountPLOJO { +public class WordCountPOJO { /** * Runs the WordCount program. @@ -50,11 +50,11 @@ public static void main(String[] args) throws Exception { // Read the text file from given input path or hard coded DataSet text = null; - try { + if(args.length >= 1) { inputPath = args[0]; - env.readTextFile(inputPath); + text = env.readTextFile(inputPath); } - catch(Exception e) { + else { System.out.println("No input file specified. Using hard coded example."); text = env.fromElements("To be", "or not to be", "or to be still", "and certainly not to be not at all", "is that the question?"); } @@ -62,22 +62,17 @@ public static void main(String[] args) throws Exception { // Split up the lines in pairs (2-tuples) containing: (word,1) DataSet words = text.flatMap(new Tokenizer()); - // Create KeySelector to be able to group CustomizedWord - CustomizedWordKeySelector keySelector = new CustomizedWordKeySelector(); - - // Instantiate customized reduce function - CustomizedWordReducer reducer = new CustomizedWordReducer(); - - // Group by the tuple field "0" and sum up tuple field "1" - DataSet result = words.groupBy(keySelector).reduce(reducer); + // Group by the tuple field "0" and sum up tuple field "1". Create KeySelector to be able to group CustomizedWord. Instantiate customized reduce function. + DataSet result = words.groupBy(new CustomizedWordKeySelector()) + .reduce(new CustomizedWordReducer()); // Print result - try { + if( args.length >= 2) { outputPath = args[1]; // write out the result result.writeAsText(outputPath); } - catch(Exception e) { + else { System.out.println("No output file specified. Printing result to console."); // Print result to console result.print(); From 05731d0296171f6104de926baea08a2bb159fc57 Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Fri, 27 Jun 2014 11:21:26 +0200 Subject: [PATCH 003/182] Renamed to make the name clearer --- .../wordcount/{WordCountPOJO.java => WordCountKeySelector.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/{WordCountPOJO.java => WordCountKeySelector.java} (99%) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPOJO.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPOJO.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java index e98325b10b063..3d0b4e4381ea6 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountPOJO.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java @@ -28,7 +28,7 @@ * over hard coded examples or text files. This example demonstrates how to use KeySelectors, ReduceFunction and FlatMapFunction. */ @SuppressWarnings("serial") -public class WordCountPOJO { +public class WordCountKeySelector { /** * Runs the WordCount program. From d394bcd2b0a7761624a21715873d01a2463c21ba Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Fri, 27 Jun 2014 11:49:40 +0200 Subject: [PATCH 004/182] Console output adopted --- .../example/java/wordcount/WordCountKeySelector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java index 3d0b4e4381ea6..08c4b77a91758 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java @@ -38,7 +38,7 @@ public class WordCountKeySelector { public static void main(String[] args) throws Exception { // Check whether arguments are given and tell user how to use this example with files. if (args.length < 2) { - System.out.println("You can specify: WordCountPLOJO , in order to work with files."); + System.out.println("You can specify: WordCountKeySelector , in order to work with files."); } // Input and output path [optional] From 806b08597222853e4c2a4204a73a638c93ac7c3f Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Sun, 6 Jul 2014 11:30:48 +0200 Subject: [PATCH 005/182] Better comments and 2014 in Licence Comments are improved in order to have an example which is as clear as possible. --- .../java/wordcount/WordCountKeySelector.java | 40 ++++++++++--------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java index 08c4b77a91758..ac6cb4beb2ffe 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java @@ -1,6 +1,6 @@ /*********************************************************************************************************************** * - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu) * * Licensed 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 @@ -25,7 +25,7 @@ /** * Implements a "WordCount" program that computes a simple word occurrence histogram - * over hard coded examples or text files. This example demonstrates how to use KeySelectors, ReduceFunction and FlatMapFunction. + * over a hard coded example or text files. This example demonstrates how to use KeySelectors, ReduceFunction and FlatMapFunction. */ @SuppressWarnings("serial") public class WordCountKeySelector { @@ -41,14 +41,14 @@ public static void main(String[] args) throws Exception { System.out.println("You can specify: WordCountKeySelector , in order to work with files."); } - // Input and output path [optional] + // Input and output path [optional]. String inputPath = null; String outputPath = null; - // Get the environment as starting point + // Get the environment as starting point. final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - // Read the text file from given input path or hard coded + // Read the text file from given input path or use a hard coded example. DataSet text = null; if(args.length >= 1) { inputPath = args[0]; @@ -59,17 +59,17 @@ public static void main(String[] args) throws Exception { text = env.fromElements("To be", "or not to be", "or to be still", "and certainly not to be not at all", "is that the question?"); } - // Split up the lines in pairs (2-tuples) containing: (word,1) + // Split up the lines in CustomizedWord containing: (word,1) DataSet words = text.flatMap(new Tokenizer()); // Group by the tuple field "0" and sum up tuple field "1". Create KeySelector to be able to group CustomizedWord. Instantiate customized reduce function. DataSet result = words.groupBy(new CustomizedWordKeySelector()) .reduce(new CustomizedWordReducer()); - // Print result + // Write result into text file if output file is specified. Otherwise print to console. if( args.length >= 2) { outputPath = args[1]; - // write out the result + // Write out the result result.writeAsText(outputPath); } else { @@ -83,15 +83,16 @@ public static void main(String[] args) throws Exception { } /** - * Implements the string tokenizer that splits sentences into words as a user-defined + * Implements a string tokenizer that splits sentences into words as a user-defined * FlatMapFunction. The function takes a line (String) and splits it into - * multiple pairs in the form of "(word,1)" CustomizedWord). + * multiple pairs in the form of CustomizedWord(word,1) ). */ public static final class Tokenizer extends FlatMapFunction { @Override public void flatMap(String value, Collector out) { - // Normalize and split the line + // Normalize (convert words to lower case, so that e.g. "Hello" + // and "hello" become the same) and split the line. String[] tokens = value.toLowerCase().split("\\W+"); // Emit the pairs @@ -104,13 +105,14 @@ public void flatMap(String value, Collector out) { } /** - * Customized reducer for CustomizedWord. + * Customized reducer for CustomizedWord. When the keys of two CustomizedWord classes are the same, + * both are reduced into one CustomizedWord class. */ public static class CustomizedWordReducer extends ReduceFunction{ /** - * This function is applied to all members of a group/ Hence to all words which to match. - * Only the count needs to be adjusted. + * This method is applied to all members of a group. Hence to all CustomizedWord instances which have + * the same key. One CustomizedWord instance is returned in which the count is increased by one. */ @Override public CustomizedWord reduce(CustomizedWord value1, @@ -123,7 +125,7 @@ public CustomizedWord reduce(CustomizedWord value1, /** * This class is a customized word and count class. It represents a Tuple with two entries (word,count). - * For this example a customized class is used in order to show how to use KeySelectors + * For this example a customized class is used in order to show how to use KeySelectors. */ public static class CustomizedWord{ // Word @@ -160,17 +162,17 @@ public String toString() { /** * KeySelector written for CustomizedWord. - * This implementation extracts the KEY out of CustomizedWord. + * This KeySelector extracts the KEY out of CustomizedWord. * */ public static class CustomizedWordKeySelector extends KeySelector { - + /** - * Extract the KEY out of CustomizedWord. + * This method is called in order to extract the KEY out of CustomizedWord. */ @Override public String getKey(CustomizedWord value) { - // Access the word (String) , which is key. + // Return the word (String), which is key. return value.word; } From 2977d7f27592154ea4885cb8214872f4f43526ba Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Tue, 8 Jul 2014 16:11:20 +0200 Subject: [PATCH 006/182] Reduce main method code. Make this example clearer by reducing the code in the main method. Not example important functionality is put into methods. --- .../java/wordcount/WordCountKeySelector.java | 197 +++++++++++------- 1 file changed, 126 insertions(+), 71 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java index ac6cb4beb2ffe..8316e6f695d69 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java @@ -21,80 +21,126 @@ import eu.stratosphere.api.java.functions.ReduceFunction; import eu.stratosphere.util.Collector; - - /** - * Implements a "WordCount" program that computes a simple word occurrence histogram - * over a hard coded example or text files. This example demonstrates how to use KeySelectors, ReduceFunction and FlatMapFunction. + * Implements a "WordCount" program that computes a simple word occurrence + * histogram over a hard coded example or text files. This example demonstrates + * how to use KeySelectors, ReduceFunction and FlatMapFunction. */ @SuppressWarnings("serial") public class WordCountKeySelector { - + /** * Runs the WordCount program. * - * @param args Input and output file. + * @param args + * Input and output file. */ public static void main(String[] args) throws Exception { - // Check whether arguments are given and tell user how to use this example with files. - if (args.length < 2) { - System.out.println("You can specify: WordCountKeySelector , in order to work with files."); - } + // Check whether arguments are given and tell user how to use this example. + printUsageIfNeeded(args); - // Input and output path [optional]. - String inputPath = null; - String outputPath = null; - - // Get the environment as starting point. - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // Read the text file from given input path or use a hard coded example. - DataSet text = null; - if(args.length >= 1) { - inputPath = args[0]; - text = env.readTextFile(inputPath); - } - else { - System.out.println("No input file specified. Using hard coded example."); - text = env.fromElements("To be", "or not to be", "or to be still", "and certainly not to be not at all", "is that the question?"); - } + // Get the environment - to be able to access files. + final ExecutionEnvironment env = ExecutionEnvironment + .getExecutionEnvironment(); + // Get input data - here the execution environment is used to read from file. + DataSet text = getTextFromInput(args, env); + // Split up the lines in CustomizedWord containing: (word,1) DataSet words = text.flatMap(new Tokenizer()); + + // Group by the tuple field "0" and sum up tuple field "1". Create + // KeySelector to be able to group CustomizedWord. Instantiate + // customized reduce function. + DataSet result = words.groupBy( + new CustomizedWordKeySelector()).reduce( + new CustomizedWordReducer()); + + // Output result, dependent on which arguments were given. + printOutput(args, result); - // Group by the tuple field "0" and sum up tuple field "1". Create KeySelector to be able to group CustomizedWord. Instantiate customized reduce function. - DataSet result = words.groupBy(new CustomizedWordKeySelector()) - .reduce(new CustomizedWordReducer()); + // Execute the defined program + env.execute("Word Count"); + } + + /** + * Handles arguments given by the user and returns the text from a specified input file + * or from a hard coded example if no file is specified. + * @param args Arguments given. + * @param env Environment of execution. + * @return Null if args is null or if args has a set input path and env is null. Otherwise it returns + * a DataSet with the file input (when input path is specified) or a hard coded example (no input path specified). + */ + private static DataSet getTextFromInput(String[] args, + ExecutionEnvironment env) { + if(args == null) + return null; + // Read the text file from given input path or use a hard coded example. + if (args.length >= 1) { + String inputPath = args[0]; + if(env != null) { + return env.readTextFile(inputPath); + }else { + return null; + } + } else { + System.out + .println("No input file specified. Using hard coded example."); + return env.fromElements("To be", "or not to be", "or to be still", + "and certainly not to be not at all", + "is that the question?"); + } + } + + /** + * Prints the result either to a file or to standard output (if no output file is specified). + * @param args Arguments given. + * @param result Result to print. + */ + private static void printOutput(String[] args, + DataSet result) { + if(result == null || args == null) + return; - // Write result into text file if output file is specified. Otherwise print to console. - if( args.length >= 2) { - outputPath = args[1]; + // print to console. + if (args.length >= 2 ) { + String outputPath = args[1]; // Write out the result result.writeAsText(outputPath); - } - else { - System.out.println("No output file specified. Printing result to console."); + } else { + System.out + .println("No output file specified. Printing result to console."); // Print result to console result.print(); } - - // Execute the defined program - env.execute("Word Count"); + } /** - * Implements a string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into - * multiple pairs in the form of CustomizedWord(word,1) ). + * Checks whether more two or more arguments are specified. If not usage pattern is printed to standard output. + * @param args Arguments given. */ - public static final class Tokenizer extends FlatMapFunction { + private static void printUsageIfNeeded(String[] args) { + if (args.length < 2) { + System.out + .println("You can specify: WordCountKeySelector , in order to work with files."); + } + } + + /** + * Implements a string tokenizer that splits sentences into words as a + * user-defined FlatMapFunction. The function takes a line (String) and + * splits it into multiple pairs in the form of CustomizedWord(word,1) ). + */ + public static final class Tokenizer extends + FlatMapFunction { @Override public void flatMap(String value, Collector out) { - // Normalize (convert words to lower case, so that e.g. "Hello" + // Normalize (convert words to lower case, so that e.g. "Hello" // and "hello" become the same) and split the line. String[] tokens = value.toLowerCase().split("\\W+"); - + // Emit the pairs for (String token : tokens) { if (token.length() > 0) { @@ -103,16 +149,19 @@ public void flatMap(String value, Collector out) { } } } - + /** - * Customized reducer for CustomizedWord. When the keys of two CustomizedWord classes are the same, - * both are reduced into one CustomizedWord class. + * Customized reducer for CustomizedWord. When the keys of two + * CustomizedWord classes are the same, both are reduced into one + * CustomizedWord class. */ - public static class CustomizedWordReducer extends ReduceFunction{ + public static class CustomizedWordReducer extends + ReduceFunction { /** - * This method is applied to all members of a group. Hence to all CustomizedWord instances which have - * the same key. One CustomizedWord instance is returned in which the count is increased by one. + * This method is applied to all members of a group. Hence to all + * CustomizedWord instances which have the same key. One CustomizedWord + * instance is returned in which the count is increased by one. */ @Override public CustomizedWord reduce(CustomizedWord value1, @@ -120,61 +169,67 @@ public CustomizedWord reduce(CustomizedWord value1, value2.count += value1.count; return value2; } - + } - + /** - * This class is a customized word and count class. It represents a Tuple with two entries (word,count). - * For this example a customized class is used in order to show how to use KeySelectors. + * This class is a customized word and count class. It represents a Tuple + * with two entries (word,count). For this example a customized class is + * used in order to show how to use KeySelectors. */ - public static class CustomizedWord{ + public static class CustomizedWord { // Word public String word; - // Count of how often word was found + // Count of how often word was found public int count; - + /** * Standard constructor. */ public CustomizedWord() { } - + /** * Constructor to set public members of class. - * @param word The word. - * @param count The number of appearances. + * + * @param word + * The word. + * @param count + * The number of appearances. */ public CustomizedWord(String word, int count) { - + // Set values this.word = word; this.count = count; } - + /** * Convert to String. For a nice printed result. */ @Override public String toString() { - return "<"+word+","+count+">"; + return "<" + word + "," + count + ">"; } } - + /** - * KeySelector written for CustomizedWord. - * This KeySelector extracts the KEY out of CustomizedWord. + * KeySelector written for CustomizedWord. This KeySelector extracts the KEY + * out of CustomizedWord. * */ - public static class CustomizedWordKeySelector extends KeySelector { - + public static class CustomizedWordKeySelector extends + KeySelector { + /** - * This method is called in order to extract the KEY out of CustomizedWord. + * This method is called in order to extract the KEY out of + * CustomizedWord. */ @Override public String getKey(CustomizedWord value) { // Return the word (String), which is key. return value.word; } - + } } From 89a6ca95366095d7798ec44d25b2d9db9684a426 Mon Sep 17 00:00:00 2001 From: TobiasWiens Date: Tue, 8 Jul 2014 16:13:03 +0200 Subject: [PATCH 007/182] Class description --- .../example/java/wordcount/WordCountKeySelector.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java index 8316e6f695d69..a88f3cb6fcde8 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountKeySelector.java @@ -31,9 +31,7 @@ public class WordCountKeySelector { /** * Runs the WordCount program. - * - * @param args - * Input and output file. + * @param args Arguments given, input and output path. */ public static void main(String[] args) throws Exception { // Check whether arguments are given and tell user how to use this example. From ae9cae0fa18541355d852164a0af4f5c20084005 Mon Sep 17 00:00:00 2001 From: Markus Holzemer Date: Mon, 12 May 2014 13:18:41 +0200 Subject: [PATCH 008/182] Disable LocalExecutor when using Client --- .../java/eu/stratosphere/client/LocalExecutor.java | 5 +++++ .../eu/stratosphere/client/program/Client.java | 6 ++++++ .../eu/stratosphere/client/program/ClientTest.java | 14 ++++++++++++++ .../api/java/ExecutionEnvironment.java | 10 ++++++++++ 4 files changed, 35 insertions(+) diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java index c5edddc5c4d11..7b540d6cf68c3 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java @@ -17,10 +17,12 @@ import org.apache.log4j.Level; +import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.JobExecutionResult; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.PlanExecutor; import eu.stratosphere.api.common.Program; +import eu.stratosphere.api.java.ExecutionEnvironment; import eu.stratosphere.client.minicluster.NepheleMiniCluster; import eu.stratosphere.compiler.DataStatistics; import eu.stratosphere.compiler.PactCompiler; @@ -66,6 +68,9 @@ public LocalExecutor() { if (System.getProperty("log4j.configuration") == null) { setLoggingLevel(Level.INFO); } + if(!ExecutionEnvironment.localExecutionIsAllowed()) { + throw new InvalidProgramException("You cannot start a job in local execution mode when submitting a job from a client."); + } } public int getJobManagerRpcPort() { diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java index 4fb65a43aec66..bb5d1f8503c76 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java @@ -66,6 +66,9 @@ public Client(InetSocketAddress jobManagerAddress, Configuration config) { configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort()); this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator(), jobManagerAddress); + + // Disable Local Execution when using a Client + ExecutionEnvironment.disableLocalExecution(); } /** @@ -90,6 +93,9 @@ public Client(Configuration config) { final InetSocketAddress jobManagerAddress = new InetSocketAddress(address, port); this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator(), jobManagerAddress); + + // Disable Local Execution when using a Client + ExecutionEnvironment.disableLocalExecution(); } public void setPrintStatusDuringExecution(boolean print) { diff --git a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java index a2f4584c7dc49..8056411c3c639 100644 --- a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java +++ b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java @@ -29,7 +29,9 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.Plan; +import eu.stratosphere.client.LocalExecutor; import eu.stratosphere.compiler.DataStatistics; import eu.stratosphere.compiler.PactCompiler; import eu.stratosphere.compiler.costs.CostEstimator; @@ -131,4 +133,16 @@ public void shouldThrowException() throws Exception verify(this.jobClientMock).submitJob(); } + + /** + * @throws Exception + */ + @Test(expected=InvalidProgramException.class) + public void tryLocalExecution() throws Exception + { + when(jobSubmissionResultMock.getReturnCode()).thenReturn(ReturnCode.ERROR); + + Client out = new Client(configMock); + LocalExecutor.execute(planMock); + } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java index 7afcfddfbf97c..355d384c3979d 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java @@ -73,6 +73,8 @@ public abstract class ExecutionEnvironment { protected List> cacheFile = new ArrayList>(); + private static boolean allowLocalExecution = true; + // -------------------------------------------------------------------------------------------- // Constructor and Properties @@ -111,6 +113,14 @@ public String getIdString() { return this.executionId.toString(); } + public static boolean localExecutionIsAllowed() { + return allowLocalExecution; + } + + public static void disableLocalExecution() { + allowLocalExecution = false; + } + // -------------------------------------------------------------------------------------------- // Data set creations // -------------------------------------------------------------------------------------------- From 43a4b293dc5f7b01768bb7aa09a2362ce6adcd39 Mon Sep 17 00:00:00 2001 From: Markus Holzemer Date: Tue, 13 May 2014 10:22:02 +0200 Subject: [PATCH 009/182] Set the disableLocalExecution() in ExecutionEnvironment to protected --- .../src/main/java/eu/stratosphere/client/program/Client.java | 4 ++-- .../eu/stratosphere/client/program/ContextEnvironment.java | 4 ++++ .../java/eu/stratosphere/api/java/ExecutionEnvironment.java | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java index bb5d1f8503c76..eb167b53173d4 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/Client.java @@ -68,7 +68,7 @@ public Client(InetSocketAddress jobManagerAddress, Configuration config) { this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator(), jobManagerAddress); // Disable Local Execution when using a Client - ExecutionEnvironment.disableLocalExecution(); + ContextEnvironment.disableLocalExecution(); } /** @@ -95,7 +95,7 @@ public Client(Configuration config) { this.compiler = new PactCompiler(new DataStatistics(), new DefaultCostEstimator(), jobManagerAddress); // Disable Local Execution when using a Client - ExecutionEnvironment.disableLocalExecution(); + ContextEnvironment.disableLocalExecution(); } public void setPrintStatusDuringExecution(boolean print) { diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java index 85d76dfc954e5..a26d6916d1e09 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java @@ -65,4 +65,8 @@ public String getExecutionPlan() throws Exception { public void setAsContext() { initializeContextEnvironment(this); } + + protected static void disableLocalExecution() { + ExecutionEnvironment.disableLocalExecution(); + } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java index 355d384c3979d..822fc15bef94e 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java @@ -117,7 +117,7 @@ public static boolean localExecutionIsAllowed() { return allowLocalExecution; } - public static void disableLocalExecution() { + protected static void disableLocalExecution() { allowLocalExecution = false; } From e998ea05bb88414312e7146ab107fca37f66cbb6 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Wed, 14 May 2014 17:16:16 +0200 Subject: [PATCH 010/182] Add comments to execution environments --- .../eu/stratosphere/client/LocalExecutor.java | 5 - .../client/program/ContextEnvironment.java | 10 +- .../api/java/ExecutionEnvironment.java | 196 ++++++++++++++++-- .../api/java/LocalEnvironment.java | 41 +++- .../api/java/RemoteEnvironment.java | 32 ++- 5 files changed, 244 insertions(+), 40 deletions(-) diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java index 7b540d6cf68c3..c5edddc5c4d11 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java @@ -17,12 +17,10 @@ import org.apache.log4j.Level; -import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.JobExecutionResult; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.PlanExecutor; import eu.stratosphere.api.common.Program; -import eu.stratosphere.api.java.ExecutionEnvironment; import eu.stratosphere.client.minicluster.NepheleMiniCluster; import eu.stratosphere.compiler.DataStatistics; import eu.stratosphere.compiler.PactCompiler; @@ -68,9 +66,6 @@ public LocalExecutor() { if (System.getProperty("log4j.configuration") == null) { setLoggingLevel(Level.INFO); } - if(!ExecutionEnvironment.localExecutionIsAllowed()) { - throw new InvalidProgramException("You cannot start a job in local execution mode when submitting a job from a client."); - } } public int getJobManagerRpcPort() { diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java index a26d6916d1e09..39bdd35d80caa 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/program/ContextEnvironment.java @@ -62,11 +62,19 @@ public String getExecutionPlan() throws Exception { return gen.getOptimizerPlanAsJSON(op); } + + @Override + public String toString() { + return "Context Environment (DOP = " + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + + ") : " + getIdString(); + } + + public void setAsContext() { initializeContextEnvironment(this); } - protected static void disableLocalExecution() { + public static void disableLocalExecution() { ExecutionEnvironment.disableLocalExecution(); } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java index 822fc15bef94e..d948d2dab6073 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java @@ -38,6 +38,7 @@ import eu.stratosphere.api.java.io.TextValueInputFormat; import eu.stratosphere.api.java.operators.DataSink; import eu.stratosphere.api.java.operators.DataSource; +import eu.stratosphere.api.java.operators.Operator; import eu.stratosphere.api.java.operators.OperatorTranslation; import eu.stratosphere.api.java.operators.translation.JavaPlan; import eu.stratosphere.api.java.tuple.Tuple2; @@ -53,49 +54,72 @@ /** - * The ExecutionEnviroment has methods to control the job execution - * and to interact with the outside world (data access). - * + * The ExecutionEnviroment is the context in which a program is executed. A + * {@link LocalEnvironment} will cause execution in the current JVM, a + * {@link RemoteEnvironment} will cause execution on a remote setup. + *

+ * The environment provides methods to control the job execution (such as + * setting the parallelism) and to interact with the outside world (data access). */ public abstract class ExecutionEnvironment { + /** The environment of the context (local by default, cluster if invoked through command line) */ private static ExecutionEnvironment contextEnvironment; + /** The default parallelism used by local environments */ private static int defaultLocalDop = Runtime.getRuntime().availableProcessors(); + /** flag to disable local executor when using the ContextEnvironment */ + private static boolean allowLocalExecution = true; + // -------------------------------------------------------------------------------------------- private final UUID executionId; private final List> sinks = new ArrayList>(); + private final List> cacheFile = new ArrayList>(); + private int degreeOfParallelism = -1; - protected List> cacheFile = new ArrayList>(); - - private static boolean allowLocalExecution = true; - // -------------------------------------------------------------------------------------------- // Constructor and Properties // -------------------------------------------------------------------------------------------- + /** + * Creates a new Execution Environment. + */ protected ExecutionEnvironment() { this.executionId = UUID.randomUUID(); } + /** + * Gets the degree of parallelism with which operation are executed by default. Operations can + * individually override this value to use a specific degree of parallelism via + * {@link Operator#setParallelism(int)}. Other operations may need to run with a different + * degree of parallelism - for example calling + * {@link DataSet#reduce(eu.stratosphere.api.java.functions.ReduceFunction)} over the entire + * set will insert eventually an operation that runs non-parallel (degree of parallelism of one). + * + * @return The degree of parallelism used by operations, unless they override that value. This method + * returns {@code -1}, if the environments default parallelism should be used. + */ public int getDegreeOfParallelism() { return degreeOfParallelism; } /** - * The degreeOfParallelism (DOP) specifies the parallelism of the operators in a job. + * Sets the degree of parallelism (DOP) for operations executed through this environment. * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with * x parallel instances. - * In a {@link LocalEnvironment} the DOP should be set to the number of hardware contexts - * On a cluster, we recommend to set the DOP + *

+ * This method overrides the default parallelism for this environment. + * The {@link LocalEnvironment} uses by default a value equal to the number of hardware + * contexts (CPU cores / threads). When executing the program via the command line client + * from a JAR file, the default degree of parallelism is the one configured for that setup. * - * @param degreeOfParallelism + * @param degreeOfParallelism The degree of parallelism */ public void setDegreeOfParallelism(int degreeOfParallelism) { if (degreeOfParallelism < 1) { @@ -105,34 +129,54 @@ public void setDegreeOfParallelism(int degreeOfParallelism) { this.degreeOfParallelism = degreeOfParallelism; } + /** + * Gets the UUID by which this environment is identified. The UUID sets the execution context + * in the cluster or local environment. + * + * @return The UUID of this environment. + * @see #getIdString() + */ public UUID getId() { return this.executionId; } + /** + * Gets the UUID by which this environment is identified, as a string. + * + * @return The UUID as a string. + * @see #getId() + */ public String getIdString() { return this.executionId.toString(); } - public static boolean localExecutionIsAllowed() { - return allowLocalExecution; - } - - protected static void disableLocalExecution() { - allowLocalExecution = false; - } - // -------------------------------------------------------------------------------------------- // Data set creations // -------------------------------------------------------------------------------------------- // ---------------------------------- Text Input Format --------------------------------------- + /** + * Creates a DataSet that represents the Strings produced by reading the given file line wise. + * The file will be read with the system's default character set. + * + * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). + * @return A DataSet that represents the data read from the given file as text lines. + */ public DataSource readTextFile(String filePath) { Validate.notNull(filePath, "The file path may not be null."); return new DataSource(this, new TextInputFormat(new Path(filePath)), BasicTypeInfo.STRING_TYPE_INFO ); } + /** + * Creates a DataSet that represents the Strings produced by reading the given file line wise. + * The {@link java.nio.charset.Charset} with the given name will be used to read the files. + * + * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). + * @param charsetName The name of the character set used to read the file. + * @return A DataSet that represents the data read from the given file as text lines. + */ public DataSource readTextFile(String filePath, String charsetName) { Validate.notNull(filePath, "The file path may not be null."); @@ -143,12 +187,37 @@ public DataSource readTextFile(String filePath, String charsetName) { // -------------------------- Text Input Format With String Value------------------------------ + /** + * Creates a DataSet that represents the Strings produced by reading the given file line wise. + * This method is similar to {@link #readTextFile(String)}, but it produces a DataSet with mutable + * {@link StringValue} objects, rather than Java Strings. StringValues can be used to tune implementations + * to be less object and garbage collection heavy. + *

+ * The file will be read with the system's default character set. + * + * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). + * @return A DataSet that represents the data read from the given file as text lines. + */ public DataSource readTextFileWithValue(String filePath) { Validate.notNull(filePath, "The file path may not be null."); return new DataSource(this, new TextValueInputFormat(new Path(filePath)), new ValueTypeInfo(StringValue.class) ); } + /** + * Creates a DataSet that represents the Strings produced by reading the given file line wise. + * This method is similar to {@link #readTextFile(String, String)}, but it produces a DataSet with mutable + * {@link StringValue} objects, rather than Java Strings. StringValues can be used to tune implementations + * to be less object and garbage collection heavy. + *

+ * The {@link java.nio.charset.Charset} with the given name will be used to read the files. + * + * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). + * @param charsetName The name of the character set used to read the file. + * @param skipInvalidLines A flag to indicate whether to skip lines that cannot be read with the given character set. + * + * @return A DataSet that represents the data read from the given file as text lines. + */ public DataSource readTextFileWithValue(String filePath, String charsetName, boolean skipInvalidLines) { Validate.notNull(filePath, "The file path may not be null."); @@ -238,7 +307,9 @@ public DataSource fromCollection(Iterator data, TypeInformation typ /** * Creates a new data set that contains the given elements. The elements must all be of the same type, - * for example, all of the Strings or integers. + * for example, all of the {@link String} or {@link Integer}. The sequence of elements must not be empty. + * Furthermore, the elements must be serializable (as defined in {@link java.io.Serializable}, because the + * execution environment may ship the elements into the cluster. * * @param data The elements to make up the data set. * @return A data set representing the given list of elements. @@ -285,7 +356,7 @@ public void registerCachedFile(String filePath, String name){ this.cacheFile.add(new Tuple2(filePath, name)); } - protected void registerCachedFiles(Plan p) throws IOException { + protected void registerCachedFilesWithPlan(Plan p) throws IOException { for (Tuple2 entry : cacheFile) { p.registerCachedFile(entry.f0, entry.f1); } @@ -308,10 +379,21 @@ public JavaPlan createProgramPlan(String jobName) { return translator.translateToPlan(this.sinks, jobName); } + /** + * Adds the given sink to this environment. Only sinks that have been added will be executed once + * the {@link #execute()} or {@link #execute(String)} method is called. + * + * @param sink The sink to add for execution. + */ void registerDataSink(DataSink sink) { this.sinks.add(sink); } + /** + * Gets a default job name, based on the timestamp when this method is invoked. + * + * @return A default job name. + */ private static String getDefaultName() { return "Stratosphere Java Job at " + Calendar.getInstance().getTime(); } @@ -320,39 +402,107 @@ private static String getDefaultName() { // Instantiation of Execution Contexts // -------------------------------------------------------------------------------------------- + /** + * Creates an execution environment that represents the context in which the program is currently executed. + * If the program is invoked standalone, this method returns a local execution environment, as returned by + * {@link #createLocalEnvironment()}. If the program is invoked from within the command line client to be + * submitted to a cluster, this method returns the execution environment of this cluster. + * + * @return The execution environment of the context in which the program is executed. + */ public static ExecutionEnvironment getExecutionEnvironment() { return contextEnvironment == null ? createLocalEnvironment() : contextEnvironment; } + /** + * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a + * multi-threaded fashion in the same JVM as the environment was created in. The default degree of + * parallelism of the local environment is the number of hardware contexts (CPU cores / threads), + * unless it was specified differently by {@link #setDefaultLocalParallelism(int)}. + * + * @return A local execution environment. + */ public static LocalEnvironment createLocalEnvironment() { return createLocalEnvironment(defaultLocalDop); } + /** + * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a + * multi-threaded fashion in the same JVM as the environment was created in. It will use the + * degree of parallelism specified in the parameter. + * + * @param degreeOfParallelism The degree of parallelism for the local environment. + * @return A local execution environment with the specified degree of parallelism. + */ public static LocalEnvironment createLocalEnvironment(int degreeOfParallelism) { LocalEnvironment lee = new LocalEnvironment(); lee.setDegreeOfParallelism(degreeOfParallelism); return lee; } + /** + * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program + * to a cluster for execution. Note that all file paths used in the program must be accessible from the + * cluster. The execution will use the cluster's default degree of parallelism, unless the parallelism is + * set explicitly via {@link ExecutionEnvironment#setDegreeOfParallelism(int)}. + * + * @param host The host name or address of the master (JobManager), where the program should be executed. + * @param port The port of the master (JobManager), where the program should be executed. + * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses + * user-defined functions, user-defined input formats, or any libraries, those must be + * provided in the JAR files. + * @return A remote environment that executes the program on a cluster. + */ public static ExecutionEnvironment createRemoteEnvironment(String host, int port, String... jarFiles) { return new RemoteEnvironment(host, port, jarFiles); } - + + /** + * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program + * to a cluster for execution. Note that all file paths used in the program must be accessible from the + * cluster. The execution will use the specified degree of parallelism. + * + * @param host The host name or address of the master (JobManager), where the program should be executed. + * @param port The port of the master (JobManager), where the program should be executed. + * @param degreeOfParallelism The degree of parallelism to use during the execution. + * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses + * user-defined functions, user-defined input formats, or any libraries, those must be + * provided in the JAR files. + * @return A remote environment that executes the program on a cluster. + */ public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles) { RemoteEnvironment rec = new RemoteEnvironment(host, port, jarFiles); rec.setDegreeOfParallelism(degreeOfParallelism); return rec; } + /** + * Sets the default parallelism that will be used for the local execution environment created by + * {@link #createLocalEnvironment()}. + * + * @param degreeOfParallelism The degree of parallelism to use as the default local parallelism. + */ public static void setDefaultLocalParallelism(int degreeOfParallelism) { defaultLocalDop = degreeOfParallelism; } + // -------------------------------------------------------------------------------------------- + // Methods to control the context and local environments for execution from packaged programs + // -------------------------------------------------------------------------------------------- + protected static void initializeContextEnvironment(ExecutionEnvironment ctx) { contextEnvironment = ctx; } - protected boolean isContextEnvironmentSet() { + protected static boolean isContextEnvironmentSet() { return contextEnvironment != null; } + + protected static boolean localExecutionIsAllowed() { + return allowLocalExecution; + } + + protected static void disableLocalExecution() { + allowLocalExecution = false; + } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/LocalEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/LocalEnvironment.java index 0e48e079628ea..2e7be725a1fff 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/LocalEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/LocalEnvironment.java @@ -16,21 +16,41 @@ import org.apache.log4j.Level; +import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.JobExecutionResult; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.PlanExecutor; import eu.stratosphere.util.LogUtils; +/** + * An {@link ExecutionEnvironment} that runs the program locally, multi-threaded, in the JVM where the + * environment is instantiated. When this environment is instantiated, it uses a default degree of parallelism + * of {@code 1}. Local environments can also be instantiated through + * {@link ExecutionEnvironment#createLocalEnvironment()} and {@link ExecutionEnvironment#createLocalEnvironment(int)}. + * The former version will pick a default degree of parallelism equal to the number of hardware contexts in the local + * machine. + */ public class LocalEnvironment extends ExecutionEnvironment { private boolean logging = false; + /** + * Creates a new local environment. + */ + public LocalEnvironment() { + if(!ExecutionEnvironment.localExecutionIsAllowed()) { + throw new InvalidProgramException("The LocalEnvironment cannot be used when submitting a program through a client."); + } + } + + // -------------------------------------------------------------------------------------------- + @Override public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); p.setDefaultParallelism(getDegreeOfParallelism()); - registerCachedFiles(p); + registerCachedFilesWithPlan(p); PlanExecutor executor = PlanExecutor.createLocalExecutor(); initLogging(); @@ -41,21 +61,36 @@ public JobExecutionResult execute(String jobName) throws Exception { public String getExecutionPlan() throws Exception { Plan p = createProgramPlan("unnamed job"); p.setDefaultParallelism(getDegreeOfParallelism()); - registerCachedFiles(p); + registerCachedFilesWithPlan(p); PlanExecutor executor = PlanExecutor.createLocalExecutor(); initLogging(); return executor.getOptimizerPlanAsJSON(p); } + // -------------------------------------------------------------------------------------------- + + /** + * Causes the local environment to print INFO level log messages to the standard error output. + */ public void enableLogging() { this.logging = true; } + /** + * Completely disables logging during the execution of programs in the local environment. + */ public void disableLogging() { this.logging = false; } + /** + * Checks whether logging during the program execution is enabled or disabled. + *

+ * By default, logging is turned off. + * + * @return True, if logging is enabled, false otherwise. + */ public boolean isLoggingEnabled() { return this.logging; } @@ -64,6 +99,8 @@ private void initLogging() { LogUtils.initializeDefaultConsoleLogger(logging ? Level.INFO : Level.OFF); } + // -------------------------------------------------------------------------------------------- + @Override public String toString() { return "Local Environment (DOP = " + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/RemoteEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/RemoteEnvironment.java index 3a63ad612b96e..3179d975b2df9 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/RemoteEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/RemoteEnvironment.java @@ -19,10 +19,17 @@ import eu.stratosphere.api.common.PlanExecutor; /** - * Special {@link ExecutionEnvironment} for running a Job - * on a remote machine. - * A host:port and jarFiles have to be supplied. - * + * An {@link ExecutionEnvironment} that sends programs + * to a cluster for execution. Note that all file paths used in the program must be accessible from the + * cluster. The execution will use the cluster's default degree of parallelism, unless the parallelism is + * set explicitly via {@link ExecutionEnvironment#setDegreeOfParallelism(int)}. + * + * @param host The host name or address of the master (JobManager), where the program should be executed. + * @param port The port of the master (JobManager), where the program should be executed. + * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses + * user-defined functions, user-defined input formats, or any libraries, those must be + * provided in the JAR files. + * @return A remote environment that executes the program on a cluster. */ public class RemoteEnvironment extends ExecutionEnvironment { @@ -32,10 +39,17 @@ public class RemoteEnvironment extends ExecutionEnvironment { private final String[] jarFiles; - + /** + * Creates a new RemoteEnvironment that points to the master (JobManager) described by the + * given host name and port. + * + * @param host The host name or address of the master (JobManager), where the program should be executed. + * @param port The port of the master (JobManager), where the program should be executed. + * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses + * user-defined functions, user-defined input formats, or any libraries, those must be + * provided in the JAR files. + */ public RemoteEnvironment(String host, int port, String... jarFiles) { - super(); - if (host == null) { throw new NullPointerException("Host must not be null."); } @@ -54,7 +68,7 @@ public RemoteEnvironment(String host, int port, String... jarFiles) { public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); p.setDefaultParallelism(getDegreeOfParallelism()); - registerCachedFiles(p); + registerCachedFilesWithPlan(p); PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, jarFiles); return executor.executePlan(p); @@ -64,7 +78,7 @@ public JobExecutionResult execute(String jobName) throws Exception { public String getExecutionPlan() throws Exception { Plan p = createProgramPlan("unnamed"); p.setDefaultParallelism(getDegreeOfParallelism()); - registerCachedFiles(p); + registerCachedFilesWithPlan(p); PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, jarFiles); return executor.getOptimizerPlanAsJSON(p); From 4eea689e3d20cd2f5cf8fa5188f05093a1f0feec Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Wed, 14 May 2014 18:03:32 +0200 Subject: [PATCH 011/182] Adjusted JVM garbage collection options to enable class unloading. --- stratosphere-dist/src/main/stratosphere-bin/bin/jobmanager.sh | 2 +- stratosphere-dist/src/main/stratosphere-bin/bin/taskmanager.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/stratosphere-dist/src/main/stratosphere-bin/bin/jobmanager.sh b/stratosphere-dist/src/main/stratosphere-bin/bin/jobmanager.sh index 54b9b22e84d10..777458c793a83 100755 --- a/stratosphere-dist/src/main/stratosphere-bin/bin/jobmanager.sh +++ b/stratosphere-dist/src/main/stratosphere-bin/bin/jobmanager.sh @@ -26,7 +26,7 @@ if [ "$EXECUTIONMODE" = "local" ]; then STRATOSPHERE_JM_HEAP=`expr $STRATOSPHERE_JM_HEAP + $STRATOSPHERE_TM_HEAP` fi -JVM_ARGS="$JVM_ARGS -Xms"$STRATOSPHERE_JM_HEAP"m -Xmx"$STRATOSPHERE_JM_HEAP"m" +JVM_ARGS="$JVM_ARGS -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:MaxPermSize=256m -Xms"$STRATOSPHERE_JM_HEAP"m -Xmx"$STRATOSPHERE_JM_HEAP"m" if [ "$STRATOSPHERE_IDENT_STRING" = "" ]; then STRATOSPHERE_IDENT_STRING="$USER" diff --git a/stratosphere-dist/src/main/stratosphere-bin/bin/taskmanager.sh b/stratosphere-dist/src/main/stratosphere-bin/bin/taskmanager.sh index 6c8bae57c0a45..3f65d3263ef4e 100755 --- a/stratosphere-dist/src/main/stratosphere-bin/bin/taskmanager.sh +++ b/stratosphere-dist/src/main/stratosphere-bin/bin/taskmanager.sh @@ -47,7 +47,7 @@ out=$STRATOSPHERE_LOG_DIR/stratosphere-$STRATOSPHERE_IDENT_STRING-taskmanager-$H pid=$STRATOSPHERE_PID_DIR/stratosphere-$STRATOSPHERE_IDENT_STRING-taskmanager.pid log_setting=(-Dlog.file="$log" -Dlog4j.configuration=file:"$STRATOSPHERE_CONF_DIR"/log4j.properties) -JVM_ARGS="$JVM_ARGS -XX:+UseParNewGC -XX:NewRatio=8 -XX:PretenureSizeThreshold=64m -Xms"$STRATOSPHERE_TM_HEAP"m -Xmx"$STRATOSPHERE_TM_HEAP"m" +JVM_ARGS="$JVM_ARGS -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:MaxPermSize=256m -XX:NewRatio=6 -Xms"$STRATOSPHERE_TM_HEAP"m -Xmx"$STRATOSPHERE_TM_HEAP"m" case $STARTSTOP in From d6959ac50f273f9c28df5dadfcd3c8e986cbd656 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Wed, 14 May 2014 18:23:40 +0200 Subject: [PATCH 012/182] Disable also local environment in client submission settings. --- .../eu/stratosphere/client/LocalExecutor.java | 6 ++++++ .../client/program/ClientTest.java | 18 ++++++++++-------- .../api/java/ExecutionEnvironment.java | 8 ++++---- 3 files changed, 20 insertions(+), 12 deletions(-) diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java index c5edddc5c4d11..89f996aab98bb 100644 --- a/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java +++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/LocalExecutor.java @@ -17,10 +17,12 @@ import org.apache.log4j.Level; +import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.JobExecutionResult; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.PlanExecutor; import eu.stratosphere.api.common.Program; +import eu.stratosphere.api.java.ExecutionEnvironment; import eu.stratosphere.client.minicluster.NepheleMiniCluster; import eu.stratosphere.compiler.DataStatistics; import eu.stratosphere.compiler.PactCompiler; @@ -63,6 +65,10 @@ public class LocalExecutor extends PlanExecutor { // -------------------------------------------------------------------------------------------- public LocalExecutor() { + if (!ExecutionEnvironment.localExecutionIsAllowed()) { + throw new InvalidProgramException("The LocalEnvironment cannot be used when submitting a program through a client."); + } + if (System.getProperty("log4j.configuration") == null) { setLoggingLevel(Level.INFO); } diff --git a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java index 8056411c3c639..479d59a6a69dd 100644 --- a/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java +++ b/stratosphere-clients/src/test/java/eu/stratosphere/client/program/ClientTest.java @@ -31,6 +31,7 @@ import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.Plan; +import eu.stratosphere.api.java.LocalEnvironment; import eu.stratosphere.client.LocalExecutor; import eu.stratosphere.compiler.DataStatistics; import eu.stratosphere.compiler.PactCompiler; @@ -134,15 +135,16 @@ public void shouldThrowException() throws Exception verify(this.jobClientMock).submitJob(); } - /** - * @throws Exception - */ + @Test(expected=InvalidProgramException.class) - public void tryLocalExecution() throws Exception - { - when(jobSubmissionResultMock.getReturnCode()).thenReturn(ReturnCode.ERROR); - - Client out = new Client(configMock); + public void tryLocalExecution() throws Exception { + new Client(configMock); LocalExecutor.execute(planMock); } + + @Test(expected=InvalidProgramException.class) + public void tryLocalEnvironmentExecution() throws Exception { + new Client(configMock); + new LocalEnvironment(); + } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java index d948d2dab6073..1513fe8737e37 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java @@ -498,11 +498,11 @@ protected static boolean isContextEnvironmentSet() { return contextEnvironment != null; } - protected static boolean localExecutionIsAllowed() { - return allowLocalExecution; - } - protected static void disableLocalExecution() { allowLocalExecution = false; } + + public static boolean localExecutionIsAllowed() { + return allowLocalExecution; + } } From 588a046c2aeb2dfa928113cde57e739f89f9e137 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 14 May 2014 12:51:10 +0200 Subject: [PATCH 013/182] TuplePairComparatorTests --- .../runtime/TuplePairComparatorTest.java | 70 ++++++++++++ .../base/TuplePairComparatorTestBase.java | 103 ++++++++++++++++++ 2 files changed, 173 insertions(+) create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java new file mode 100644 index 0000000000000..d0a8e899a0a95 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java @@ -0,0 +1,70 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.base.DoubleComparator; +import eu.stratosphere.api.common.typeutils.base.IntComparator; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.api.java.typeutils.runtime.tuple.base.TuplePairComparatorTestBase; + +public class TuplePairComparatorTest extends TuplePairComparatorTestBase, Tuple3> { + + @SuppressWarnings("unchecked") + Tuple3[] dataISD = new Tuple3[]{ + new Tuple3(4, "hello", 20.0), + new Tuple3(4, "world", 23.2), + new Tuple3(5, "hello", 20.0), + new Tuple3(5, "world", 23.2), + new Tuple3(6, "hello", 20.0), + new Tuple3(6, "world", 23.2), + new Tuple3(7, "hello", 20.0), + new Tuple3(7, "world", 23.2) + }; + + @SuppressWarnings("unchecked") + Tuple3[] dataIDL = new Tuple3[]{ + new Tuple3(4, 20.0, new Long(14)), + new Tuple3(4, 23.2, new Long(15)), + new Tuple3(5, 20.0, new Long(15)), + new Tuple3(5, 23.2, new Long(20)), + new Tuple3(6, 20.0, new Long(20)), + new Tuple3(6, 23.2, new Long(29)), + new Tuple3(7, 20.0, new Long(29)), + new Tuple3(7, 23.2, new Long(34)) + }; + + @Override + protected TuplePairComparator, Tuple3> createComparator(boolean ascending) { + return new TuplePairComparator, Tuple3>( + new int[]{0, 2}, + new int[]{0, 1}, + new TypeComparator[]{ + new IntComparator(ascending), + new DoubleComparator(ascending) + }, + new TypeComparator[]{ + new IntComparator(ascending), + new DoubleComparator(ascending) + } + ); + } + + @Override + protected Tuple2[], Tuple3[]> getSortedTestData() { + return new Tuple2(dataISD, dataIDL); + } +} diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java new file mode 100644 index 0000000000000..e886b6fde3c10 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java @@ -0,0 +1,103 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime.tuple.base; + +import eu.stratosphere.api.java.tuple.Tuple; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.api.java.typeutils.runtime.TuplePairComparator; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import org.junit.Test; + +/** + * Abstract test base for TuplePairComparators. + * + * @param + * @param + */ +public abstract class TuplePairComparatorTestBase { + + protected abstract TuplePairComparator createComparator(boolean ascending); + + protected abstract Tuple2 getSortedTestData(); + + @Test + public void testEqualityWithReference() { + try { + TuplePairComparator comparator = getComparator(true); + Tuple2 data = getSortedData(); + for (int x = 0; x < data.f0.length; x++) { + comparator.setReference(data.f0[x]); + + assertTrue(comparator.equalToReference(data.f1[x])); + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testInequalityWithReference() { + testGreatSmallAscDescWithReference(true); + testGreatSmallAscDescWithReference(false); + } + + protected void testGreatSmallAscDescWithReference(boolean ascending) { + try { + Tuple2 data = getSortedData(); + + TuplePairComparator comparator = getComparator(ascending); + + //compares every element in high with every element in low + for (int x = 0; x < data.f0.length - 1; x++) { + for (int y = x + 1; y < data.f1.length; y++) { + comparator.setReference(data.f0[x]); + if (ascending) { + assertTrue(comparator.compareToReference(data.f1[y]) > 0); + } else { + assertTrue(comparator.compareToReference(data.f1[y]) < 0); + } + } + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + protected TuplePairComparator getComparator(boolean ascending) { + TuplePairComparator comparator = createComparator(ascending); + if (comparator == null) { + throw new RuntimeException("Test case corrupt. Returns null as comparator."); + } + return comparator; + } + + protected Tuple2 getSortedData() { + Tuple2 data = getSortedTestData(); + if (data == null || data.f0 == null || data.f1 == null) { + throw new RuntimeException("Test case corrupt. Returns null as test data."); + } + if (data.f0.length < 2 || data.f1.length < 2) { + throw new RuntimeException("Test case does not provide enough sorted test data."); + } + + return data; + } +} From 552aea9c76245800be1feb63a3997dfee2728d32 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Wed, 14 May 2014 18:52:53 +0200 Subject: [PATCH 014/182] Add tests for TupleLeadingFieldComparator --- .../TupleLeadingFieldPairComparatorTest.java | 60 +++++++++++++++++++ .../runtime/TuplePairComparatorTest.java | 46 +++++++------- .../base/TuplePairComparatorTestBase.java | 13 ++-- 3 files changed, 91 insertions(+), 28 deletions(-) create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java new file mode 100644 index 0000000000000..6c77ffa996d81 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java @@ -0,0 +1,60 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import eu.stratosphere.api.common.typeutils.TypePairComparator; +import eu.stratosphere.api.common.typeutils.base.IntComparator; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.api.java.tuple.Tuple4; +import eu.stratosphere.api.java.typeutils.runtime.tuple.base.TuplePairComparatorTestBase; + +public class TupleLeadingFieldPairComparatorTest extends TuplePairComparatorTestBase, Tuple4> { + + @SuppressWarnings("unchecked") + private Tuple3[] dataISD = new Tuple3[]{ + new Tuple3(4, "hello", 20.0), + new Tuple3(5, "world", 23.2), + new Tuple3(6, "hello", 18.0), + new Tuple3(7, "world", 19.2), + new Tuple3(8, "hello", 16.0), + new Tuple3(9, "world", 17.2), + new Tuple3(10, "hello", 14.0), + new Tuple3(11, "world", 15.2) + }; + + @SuppressWarnings("unchecked") + private Tuple4[] dataIDL = new Tuple4[]{ + new Tuple4(4, 0.11f, 14L, 20.0), + new Tuple4(5, 0.221f, 15L, 23.2), + new Tuple4(6, 0.33f, 15L, 18.0), + new Tuple4(7, 0.44f, 20L, 19.2), + new Tuple4(8, 0.55f, 20L, 16.0), + new Tuple4(9, 0.66f, 29L, 17.2), + new Tuple4(10, 0.77f, 29L, 14.0), + new Tuple4(11, 0.88f, 34L, 15.2) + }; + + @Override + protected TypePairComparator, Tuple4> createComparator(boolean ascending) { + return new TupleLeadingFieldPairComparator, Tuple4>( + new IntComparator(ascending), new IntComparator(ascending)); + } + + @Override + protected Tuple2[], Tuple4[]> getSortedTestData() { + return new Tuple2[], Tuple4[]>(dataISD, dataIDL); + } +} diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java index d0a8e899a0a95..410562587391e 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TuplePairComparatorTest.java @@ -19,39 +19,41 @@ import eu.stratosphere.api.common.typeutils.base.IntComparator; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.api.java.tuple.Tuple4; import eu.stratosphere.api.java.typeutils.runtime.tuple.base.TuplePairComparatorTestBase; -public class TuplePairComparatorTest extends TuplePairComparatorTestBase, Tuple3> { +public class TuplePairComparatorTest extends TuplePairComparatorTestBase, Tuple4> { @SuppressWarnings("unchecked") - Tuple3[] dataISD = new Tuple3[]{ + private Tuple3[] dataISD = new Tuple3[]{ new Tuple3(4, "hello", 20.0), new Tuple3(4, "world", 23.2), - new Tuple3(5, "hello", 20.0), - new Tuple3(5, "world", 23.2), - new Tuple3(6, "hello", 20.0), - new Tuple3(6, "world", 23.2), - new Tuple3(7, "hello", 20.0), - new Tuple3(7, "world", 23.2) + new Tuple3(5, "hello", 18.0), + new Tuple3(5, "world", 19.2), + new Tuple3(6, "hello", 16.0), + new Tuple3(6, "world", 17.2), + new Tuple3(7, "hello", 14.0), + new Tuple3(7, "world", 15.2) }; @SuppressWarnings("unchecked") - Tuple3[] dataIDL = new Tuple3[]{ - new Tuple3(4, 20.0, new Long(14)), - new Tuple3(4, 23.2, new Long(15)), - new Tuple3(5, 20.0, new Long(15)), - new Tuple3(5, 23.2, new Long(20)), - new Tuple3(6, 20.0, new Long(20)), - new Tuple3(6, 23.2, new Long(29)), - new Tuple3(7, 20.0, new Long(29)), - new Tuple3(7, 23.2, new Long(34)) + private Tuple4[] dataIDL = new Tuple4[]{ + new Tuple4(4, 0.11f, 14L, 20.0), + new Tuple4(4, 0.221f, 15L, 23.2), + new Tuple4(5, 0.33f, 15L, 18.0), + new Tuple4(5, 0.44f, 20L, 19.2), + new Tuple4(6, 0.55f, 20L, 16.0), + new Tuple4(6, 0.66f, 29L, 17.2), + new Tuple4(7, 0.77f, 29L, 14.0), + new Tuple4(7, 0.88f, 34L, 15.2) }; + @SuppressWarnings("unchecked") @Override - protected TuplePairComparator, Tuple3> createComparator(boolean ascending) { - return new TuplePairComparator, Tuple3>( + protected TuplePairComparator, Tuple4> createComparator(boolean ascending) { + return new TuplePairComparator, Tuple4>( new int[]{0, 2}, - new int[]{0, 1}, + new int[]{0, 3}, new TypeComparator[]{ new IntComparator(ascending), new DoubleComparator(ascending) @@ -64,7 +66,7 @@ protected TuplePairComparator, Tuple3[], Tuple3[]> getSortedTestData() { - return new Tuple2(dataISD, dataIDL); + protected Tuple2[], Tuple4[]> getSortedTestData() { + return new Tuple2[], Tuple4[]>(dataISD, dataIDL); } } diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java index e886b6fde3c10..f246e0223a97a 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java @@ -14,11 +14,12 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.typeutils.runtime.tuple.base; +import eu.stratosphere.api.common.typeutils.TypePairComparator; import eu.stratosphere.api.java.tuple.Tuple; import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.api.java.typeutils.runtime.TuplePairComparator; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; + import org.junit.Test; /** @@ -29,14 +30,14 @@ */ public abstract class TuplePairComparatorTestBase { - protected abstract TuplePairComparator createComparator(boolean ascending); + protected abstract TypePairComparator createComparator(boolean ascending); protected abstract Tuple2 getSortedTestData(); @Test public void testEqualityWithReference() { try { - TuplePairComparator comparator = getComparator(true); + TypePairComparator comparator = getComparator(true); Tuple2 data = getSortedData(); for (int x = 0; x < data.f0.length; x++) { comparator.setReference(data.f0[x]); @@ -60,7 +61,7 @@ protected void testGreatSmallAscDescWithReference(boolean ascending) { try { Tuple2 data = getSortedData(); - TuplePairComparator comparator = getComparator(ascending); + TypePairComparator comparator = getComparator(ascending); //compares every element in high with every element in low for (int x = 0; x < data.f0.length - 1; x++) { @@ -81,8 +82,8 @@ protected void testGreatSmallAscDescWithReference(boolean ascending) { } // -------------------------------------------------------------------------------------------- - protected TuplePairComparator getComparator(boolean ascending) { - TuplePairComparator comparator = createComparator(ascending); + protected TypePairComparator getComparator(boolean ascending) { + TypePairComparator comparator = createComparator(ascending); if (comparator == null) { throw new RuntimeException("Test case corrupt. Returns null as comparator."); } From 52f6ff77f97a005ff799eff13da00833020c63f7 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 14 May 2014 15:03:08 +0200 Subject: [PATCH 015/182] Fix for branching merging logic in BulkIterationPlanNode --- .../compiler/plan/BulkIterationPlanNode.java | 7 ++++ .../compiler/BranchingPlansCompilerTest.java | 36 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/BulkIterationPlanNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/BulkIterationPlanNode.java index c9cbb7dcd454b..07b434b80e909 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/BulkIterationPlanNode.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plan/BulkIterationPlanNode.java @@ -15,6 +15,9 @@ import static eu.stratosphere.compiler.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; import static eu.stratosphere.compiler.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; + +import java.util.HashMap; + import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; import eu.stratosphere.compiler.CompilerException; import eu.stratosphere.compiler.costs.Costs; @@ -150,6 +153,10 @@ private void mergeBranchPlanMaps() { for(OptimizerNode.UnclosedBranchDescriptor desc: template.getOpenBranches()){ OptimizerNode brancher = desc.getBranchingNode(); + if(branchPlan == null) { + branchPlan = new HashMap(6); + } + if(!branchPlan.containsKey(brancher)){ PlanNode selectedCandidate = null; diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java index d39e7f94e3f25..2d082f3aa7504 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java @@ -665,6 +665,42 @@ public void testBranchAfterIteration() { Assert.fail(e.getMessage()); } } + + @Test + public void testBranchBeforeIteration() { + FileDataSource source1 = new FileDataSource(DummyInputFormat.class, IN_FILE, "Source 1"); + FileDataSource source2 = new FileDataSource(DummyInputFormat.class, IN_FILE, "Source 2"); + + BulkIteration iteration = new BulkIteration("Loop"); + iteration.setInput(source2); + iteration.setMaximumNumberOfIterations(10); + + MapOperator inMap = MapOperator.builder(new IdentityMap()) + .input(source1) + .name("In Iteration Map") + .setBroadcastVariable("BC", iteration.getPartialSolution()) + .build(); + + iteration.setNextPartialSolution(inMap); + + MapOperator postMap = MapOperator.builder(new IdentityMap()) + .input(source1) + .name("Post Iteration Map") + .setBroadcastVariable("BC", iteration) + .build(); + + FileDataSink sink = new FileDataSink(DummyOutputFormat.class, OUT_FILE, postMap, "Sink"); + + Plan plan = new Plan(sink); + + try { + compileNoStats(plan); + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + } /** * Test to ensure that sourceA is inside as well as outside of the iteration the same From e68a6b0809a73a35c35cc3e4c44d300f5dd6a752 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 12 May 2014 14:08:09 +0200 Subject: [PATCH 016/182] Moved record API example jobs to stratosphere-tests --- .../generator}/KMeansSampleDataGenerator.java | 2 +- .../relational/generator/WebLogGenerator.java | 2 +- .../KMeansIterativeNepheleITCase.java | 12 ++++----- .../test/cancelling/MapCancelingITCase.java | 4 +-- .../cancelling/MatchJoinCancelingITCase.java | 8 +++--- .../clients/examples/LocalExecutorITCase.java | 2 +- .../examples/KMeansSingleStepTest.java | 2 +- .../examples/RelationalQueryCompilerTest.java | 2 +- .../examples/WordCountCompilerTest.java | 6 ++--- .../ConnectedComponentsCoGroupTest.java | 2 +- .../iterations/ConnectedComponentsTest.java | 8 +++--- .../iterations/IterativeKMeansTest.java | 7 +++-- .../plandump/DumpCompiledPlanTest.java | 10 +++---- .../plandump/PreviewPlanDumpTest.java | 10 +++---- .../ComputeEdgeDegreesITCase.java | 2 +- ...EnumTrianglesOnEdgesWithDegreesITCase.java | 2 +- .../RelationalQueryITCase.java | 2 +- .../WebLogAnalysisITCase.java | 2 +- .../exampleScalaPrograms/WordCountITCase.java | 2 +- .../WordCountPactValueITCase.java | 2 +- .../WordCountWithCountFunctionITCase.java | 2 +- .../CoGroupConnectedComponentsITCase.java | 4 +-- .../iterative/ConnectedComponentsITCase.java | 2 +- ...tedComponentsWithDeferredUpdateITCase.java | 6 ++--- ...dComponentsWithSolutionSetFirstITCase.java | 6 ++--- .../iterative/DanglingPageRankITCase.java | 4 +-- .../test/iterative/DeltaPageRankITCase.java | 2 +- .../IterationWithChainingITCase.java | 20 +++++++------- .../iterative/IterationWithUnionITCase.java | 6 ++--- .../test/iterative/IterativeKMeansITCase.java | 2 +- .../KMeansTutorialExampleITCase.java | 2 +- .../test/iterative/PageRankITCase.java | 4 +-- .../ConnectedComponentsNepheleITCase.java | 6 ++--- .../IterationWithChainingNepheleITCase.java | 26 +++++++++++-------- .../CollectionSourceTest.java | 2 +- .../CollectionValidationTest.java | 2 +- .../ComputeEdgeDegreesITCase.java | 6 ++--- ...EnumTrianglesOnEdgesWithDegreesITCase.java | 6 ++--- .../EnumTrianglesRDFITCase.java | 4 +-- .../GlobalSortingITCase.java | 2 +- .../GlobalSortingMixedOrderITCase.java | 2 +- .../GroupOrderReduceITCase.java | 2 +- .../KMeansStepITCase.java | 4 +-- .../MergeOnlyJoinITCase.java | 4 +-- .../PairwiseSPITCase.java | 13 +++++----- .../TPCHQuery10ITCase.java | 4 +-- .../TPCHQuery3ITCase.java | 6 ++--- .../TPCHQuery3WithUnionITCase.java | 4 +-- .../TPCHQuery4ITCase.java | 4 +-- .../TPCHQuery9ITCase.java | 4 +-- .../TPCHQueryAsterixITCase.java | 4 +-- .../TeraSortITCase.java | 6 ++--- .../WebLogAnalysisITCase.java | 4 +-- .../WordCountITCase.java | 4 +-- .../WordCountUnionReduceITCase.java | 6 ++--- .../recordJobs/graph}/ComputeEdgeDegrees.java | 6 ++--- .../ConnectedComponentsWithCoGroup.java | 6 ++--- .../recordJobs/graph}/DanglingPageRank.java | 9 ++++++- .../DeltaPageRankWithInitialDeltas.java | 2 +- .../EnumTrianglesOnEdgesWithDegrees.java | 6 ++--- .../graph}/EnumTrianglesRdfFoaf.java | 2 +- .../graph}/EnumTrianglesWithDegrees.java | 20 +++++++------- .../test/recordJobs/graph}/PairwiseSP.java | 2 +- .../recordJobs/graph}/SimplePageRank.java | 6 ++++- .../graph}/WorksetConnectedComponents.java | 2 +- .../pageRankUtil}/AsciiLongArrayView.java | 2 +- .../DanglingPageRankInputFormat.java | 4 +-- .../DiffL1NormConvergenceCriterion.java | 2 +- .../pageRankUtil}/DotProductCoGroup.java | 4 +-- .../graph/pageRankUtil}/DotProductMatch.java | 2 +- .../ImprovedAdjacencyListInputFormat.java | 2 +- .../graph/pageRankUtil}/LongArrayView.java | 2 +- .../graph/pageRankUtil}/PageRankStats.java | 2 +- .../PageRankStatsAggregator.java | 2 +- .../pageRankUtil}/PageWithRankOutFormat.java | 2 +- .../triangleEnumUtil}/EdgeInputFormat.java | 2 +- .../EdgeWithDegreesInputFormat.java | 2 +- .../EdgeWithDegreesOutputFormat.java | 2 +- .../TriangleOutputFormat.java | 2 +- .../test/recordJobs}/kmeans/KMeans.java | 2 +- .../recordJobs}/kmeans/KMeansIterative.java | 12 ++++----- .../kmeans}/KMeansSingleStep.java | 10 +++---- .../kmeans/KMeansTutorialExample.java | 2 +- .../kmeans/udfs/ComputeDistance.java | 2 +- .../udfs/ComputeDistanceParameterized.java | 2 +- .../recordJobs}/kmeans/udfs/CoordVector.java | 2 +- .../kmeans/udfs/FindNearestCenter.java | 2 +- .../kmeans/udfs/PointInFormat.java | 2 +- .../kmeans/udfs/PointOutFormat.java | 2 +- .../kmeans/udfs/RecomputeClusterCenter.java | 2 +- .../relational}/MergeOnlyJoin.java | 2 +- .../relational}/TPCHQuery1.java | 8 +++--- .../relational}/TPCHQuery10.java | 6 ++--- .../recordJobs}/relational/TPCHQuery3.java | 2 +- .../relational}/TPCHQuery3Unioned.java | 9 +++---- .../relational}/TPCHQuery4.java | 8 +++--- .../relational}/TPCHQuery9.java | 18 +++++++++++-- .../relational}/TPCHQueryAsterix.java | 2 +- .../relational/WebLogAnalysis.java | 2 +- .../query1Util}/GroupByReturnFlag.java | 4 +-- .../query1Util}/LineItemFilter.java | 4 +-- .../query1Util}/LineItemFilterTest.java | 4 +-- .../query9Util}/AmountAggregate.java | 2 +- .../query9Util}/FilteredPartsJoin.java | 4 +-- .../relational/query9Util}/IntPair.java | 2 +- .../relational/query9Util}/LineItemMap.java | 4 +-- .../relational/query9Util}/OrderMap.java | 4 +-- .../query9Util}/OrderedPartsJoin.java | 4 +-- .../relational/query9Util}/PartFilter.java | 4 +-- .../relational/query9Util}/PartJoin.java | 4 +-- .../relational/query9Util}/PartListJoin.java | 2 +- .../relational/query9Util}/PartsuppMap.java | 4 +-- .../relational/query9Util}/StringIntPair.java | 2 +- .../StringIntPairStringDataOutFormat.java | 2 +- .../relational/query9Util}/SupplierMap.java | 4 +-- .../relational/query9Util}/SuppliersJoin.java | 4 +-- .../recordJobs}/sort/ReduceGroupSort.java | 2 +- .../test/recordJobs}/sort/TeraSort.java | 10 +++---- .../sort/tsUtil}/TeraDistribution.java | 2 +- .../sort/tsUtil}/TeraInputFormat.java | 2 +- .../test/recordJobs/sort/tsUtil}/TeraKey.java | 2 +- .../sort/tsUtil}/TeraOutputFormat.java | 2 +- .../recordJobs/sort/tsUtil}/TeraValue.java | 2 +- .../test/recordJobs}/util/ConfigUtils.java | 2 +- .../util/DiscardingOutputFormat.java | 2 +- .../util/InfiniteIntegerInputFormat.java | 2 +- .../InfiniteIntegerInputFormatWithDelay.java | 2 +- .../util/IntTupleDataInFormat.java | 2 +- .../util/StringTupleDataOutFormat.java | 2 +- .../util/Tuple.java | 2 +- .../util/UniformIntInput.java | 2 +- .../wordcount/AnonymousWordCount.java | 2 +- .../test/recordJobs}/wordcount/WordCount.java | 2 +- .../wordcount/WordCountAccumulators.java | 2 +- .../wordcount/WordCountOptimized.java | 2 +- .../util/tests/IntTupleDataInFormatTest.java | 4 +-- .../testPrograms/util/tests/TupleTest.java | 2 +- 137 files changed, 304 insertions(+), 270 deletions(-) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/{record/kmeans => clustering/generator}/KMeansSampleDataGenerator.java (98%) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/{record => }/relational/generator/WebLogGenerator.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/CollectionSourceTest.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/CollectionValidationTest.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/ComputeEdgeDegreesITCase.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/EnumTrianglesOnEdgesWithDegreesITCase.java (94%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/EnumTrianglesRDFITCase.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/GlobalSortingITCase.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/GlobalSortingMixedOrderITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/GroupOrderReduceITCase.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/KMeansStepITCase.java (94%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/MergeOnlyJoinITCase.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/PairwiseSPITCase.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TPCHQuery10ITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TPCHQuery3ITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TPCHQuery3WithUnionITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TPCHQuery4ITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TPCHQuery9ITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TPCHQueryAsterixITCase.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/TeraSortITCase.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/WebLogAnalysisITCase.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/WordCountITCase.java (93%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{exampleRecordPrograms => recordJobTests}/WordCountUnionReduceITCase.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/ComputeEdgeDegrees.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs/graph}/ConnectedComponentsWithCoGroup.java (95%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/DanglingPageRank.java (86%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/incremental/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/DeltaPageRankWithInitialDeltas.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/EnumTrianglesOnEdgesWithDegrees.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/EnumTrianglesRdfFoaf.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/EnumTrianglesWithDegrees.java (82%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/shortestpaths => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/PairwiseSP.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph}/SimplePageRank.java (94%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs/graph}/WorksetConnectedComponents.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/AsciiLongArrayView.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/DanglingPageRankInputFormat.java (95%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/DiffL1NormConvergenceCriterion.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/DotProductCoGroup.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/DotProductMatch.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/ImprovedAdjacencyListInputFormat.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/LongArrayView.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/PageRankStats.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/PageRankStatsAggregator.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil}/PageWithRankOutFormat.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil}/EdgeInputFormat.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil}/EdgeWithDegreesInputFormat.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil}/EdgeWithDegreesOutputFormat.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil}/TriangleOutputFormat.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/KMeans.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/KMeansIterative.java (92%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs/kmeans}/KMeansSingleStep.java (91%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/KMeansTutorialExample.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/ComputeDistance.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/ComputeDistanceParameterized.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/CoordVector.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/FindNearestCenter.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/PointInFormat.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/PointOutFormat.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/kmeans/udfs/RecomputeClusterCenter.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/mergeOnlyJoin => recordJobs/relational}/MergeOnlyJoin.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch1 => recordJobs/relational}/TPCHQuery1.java (89%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch10 => recordJobs/relational}/TPCHQuery10.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/relational/TPCHQuery3.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch3Unioned => recordJobs/relational}/TPCHQuery3Unioned.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch4 => recordJobs/relational}/TPCHQuery4.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational}/TPCHQuery9.java (88%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs/relational}/TPCHQueryAsterix.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/relational/WebLogAnalysis.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch1 => recordJobs/relational/query1Util}/GroupByReturnFlag.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch1 => recordJobs/relational/query1Util}/LineItemFilter.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch1 => recordJobs/relational/query1Util}/LineItemFilterTest.java (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/AmountAggregate.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/FilteredPartsJoin.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/IntPair.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/LineItemMap.java (94%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/OrderMap.java (93%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/OrderedPartsJoin.java (94%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/PartFilter.java (93%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/PartJoin.java (94%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/PartListJoin.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/PartsuppMap.java (93%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/StringIntPair.java (95%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/StringIntPairStringDataOutFormat.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/SupplierMap.java (93%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms/tpch9 => recordJobs/relational/query9Util}/SuppliersJoin.java (93%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/sort/ReduceGroupSort.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/sort/TeraSort.java (90%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil}/TeraDistribution.java (97%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil}/TeraInputFormat.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil}/TeraKey.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil}/TeraOutputFormat.java (96%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil}/TeraValue.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/util/ConfigUtils.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/DiscardingOutputFormat.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/InfiniteIntegerInputFormat.java (96%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/InfiniteIntegerInputFormatWithDelay.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/IntTupleDataInFormat.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/StringTupleDataOutFormat.java (97%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/Tuple.java (99%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{testPrograms => recordJobs}/util/UniformIntInput.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/wordcount/AnonymousWordCount.java (98%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/wordcount/WordCount.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/wordcount/WordCountAccumulators.java (99%) rename {stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record => stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs}/wordcount/WordCountOptimized.java (99%) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansSampleDataGenerator.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/generator/KMeansSampleDataGenerator.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansSampleDataGenerator.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/generator/KMeansSampleDataGenerator.java index 22e2ffdfffe12..37c2f1e95fc17 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansSampleDataGenerator.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/generator/KMeansSampleDataGenerator.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.kmeans; +package eu.stratosphere.example.java.clustering.generator; import java.io.BufferedWriter; import java.io.File; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/generator/WebLogGenerator.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/generator/WebLogGenerator.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/generator/WebLogGenerator.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/generator/WebLogGenerator.java index d7a59c7ea2b0f..7a3c0236a6462 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/generator/WebLogGenerator.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/generator/WebLogGenerator.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.relational.generator; +package eu.stratosphere.example.java.relational.generator; import java.io.FileWriter; import java.io.IOException; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java index 83593af9e9f00..e1d65799e5ae4 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java @@ -14,7 +14,6 @@ **********************************************************************************************************************/ package eu.stratosphere.test.broadcastvars; -import eu.stratosphere.test.util.RecordAPITestBase; import org.apache.log4j.Level; import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper; @@ -23,10 +22,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat; import eu.stratosphere.configuration.Configuration; import eu.stratosphere.core.fs.Path; -import eu.stratosphere.example.java.record.kmeans.KMeans.PointBuilder; -import eu.stratosphere.example.java.record.kmeans.KMeans.PointOutFormat; -import eu.stratosphere.example.java.record.kmeans.KMeans.RecomputeClusterCenter; -import eu.stratosphere.example.java.record.kmeans.KMeans.SelectNearestCenter; import eu.stratosphere.nephele.io.DistributionPattern; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobGraph; @@ -40,15 +35,20 @@ import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparatorFactory; import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordSerializerFactory; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; -import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.CollectorMapDriver; +import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.NoOpDriver; import eu.stratosphere.pact.runtime.task.GroupReduceDriver; import eu.stratosphere.pact.runtime.task.chaining.ChainedCollectorMapDriver; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.pact.runtime.task.util.TaskConfig; import eu.stratosphere.test.iterative.nephele.JobGraphUtils; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointBuilder; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.RecomputeClusterCenter; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.SelectNearestCenter; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointOutFormat; import eu.stratosphere.test.testdata.KMeansData; +import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.util.LogUtils; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java index f94d2fd43b87f..f12edbbc2b51c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MapCancelingITCase.java @@ -21,8 +21,8 @@ import eu.stratosphere.api.java.record.functions.MapFunction; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.util.DiscardingOutputFormat; -import eu.stratosphere.test.testPrograms.util.InfiniteIntegerInputFormat; +import eu.stratosphere.test.recordJobs.util.DiscardingOutputFormat; +import eu.stratosphere.test.recordJobs.util.InfiniteIntegerInputFormat; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java index 4c20954e9f1c1..6e1c32acc1ab5 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/cancelling/MatchJoinCancelingITCase.java @@ -21,10 +21,10 @@ import eu.stratosphere.api.java.record.functions.JoinFunction; import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.util.DiscardingOutputFormat; -import eu.stratosphere.test.testPrograms.util.InfiniteIntegerInputFormat; -import eu.stratosphere.test.testPrograms.util.InfiniteIntegerInputFormatWithDelay; -import eu.stratosphere.test.testPrograms.util.UniformIntInput; +import eu.stratosphere.test.recordJobs.util.DiscardingOutputFormat; +import eu.stratosphere.test.recordJobs.util.InfiniteIntegerInputFormat; +import eu.stratosphere.test.recordJobs.util.InfiniteIntegerInputFormatWithDelay; +import eu.stratosphere.test.recordJobs.util.UniformIntInput; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java index c0db1940d2893..802fd0b5c32e3 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java @@ -20,7 +20,7 @@ import org.junit.Test; import eu.stratosphere.client.LocalExecutor; -import eu.stratosphere.example.java.record.wordcount.WordCount; +import eu.stratosphere.test.recordJobs.wordcount.WordCount; import eu.stratosphere.test.testdata.WordCountData; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/KMeansSingleStepTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/KMeansSingleStepTest.java index aef17cfa94f56..55cce3be10191 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/KMeansSingleStepTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/KMeansSingleStepTest.java @@ -29,7 +29,7 @@ import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; -import eu.stratosphere.test.testPrograms.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; public class KMeansSingleStepTest extends CompilerTestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/RelationalQueryCompilerTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/RelationalQueryCompilerTest.java index d038b38141a2c..983100294aea0 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/RelationalQueryCompilerTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/RelationalQueryCompilerTest.java @@ -26,11 +26,11 @@ import eu.stratosphere.compiler.plan.OptimizedPlan; import eu.stratosphere.compiler.plan.SingleInputPlanNode; import eu.stratosphere.compiler.plan.SinkPlanNode; -import eu.stratosphere.example.java.record.relational.TPCHQuery3; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; /** * Tests TPCH Q3 (simplified) under various input conditions. diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/WordCountCompilerTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/WordCountCompilerTest.java index 7e5dd9035f8f3..9eebce5b2fb58 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/WordCountCompilerTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/examples/WordCountCompilerTest.java @@ -32,13 +32,13 @@ import eu.stratosphere.compiler.plan.OptimizedPlan; import eu.stratosphere.compiler.plan.SingleInputPlanNode; import eu.stratosphere.compiler.plan.SinkPlanNode; -import eu.stratosphere.example.java.record.wordcount.WordCount; -import eu.stratosphere.example.java.record.wordcount.WordCount.CountWords; -import eu.stratosphere.example.java.record.wordcount.WordCount.TokenizeLine; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; +import eu.stratosphere.test.recordJobs.wordcount.WordCount; +import eu.stratosphere.test.recordJobs.wordcount.WordCount.CountWords; +import eu.stratosphere.test.recordJobs.wordcount.WordCount.TokenizeLine; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsCoGroupTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsCoGroupTest.java index fa570a2245793..346f849ca8cfa 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsCoGroupTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsCoGroupTest.java @@ -29,7 +29,7 @@ import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; -import eu.stratosphere.test.testPrograms.ConnectedComponentsWithCoGroup; +import eu.stratosphere.test.recordJobs.graph.ConnectedComponentsWithCoGroup; /** * diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsTest.java index be24b6fc36101..1e415da5803cd 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/ConnectedComponentsTest.java @@ -42,10 +42,10 @@ import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.DuplicateLongMap; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.MinimumComponentIDReduce; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; import eu.stratosphere.types.LongValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java index 5757383d0f631..d2b2965d8613f 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java @@ -12,7 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.test.compiler.iterations; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.util.Arrays; @@ -27,11 +30,11 @@ import eu.stratosphere.compiler.plan.SingleInputPlanNode; import eu.stratosphere.compiler.plan.SinkPlanNode; import eu.stratosphere.compiler.plantranslate.NepheleJobGraphGenerator; -import eu.stratosphere.example.java.record.kmeans.KMeans; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; +import eu.stratosphere.test.recordJobs.kmeans.KMeans; public class IterativeKMeansTest extends CompilerTestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java index e61c3ae6f180b..2064b3f6ff67d 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java @@ -25,12 +25,12 @@ import eu.stratosphere.compiler.plan.OptimizedPlan; import eu.stratosphere.compiler.plandump.PlanJSONDumpGenerator; import eu.stratosphere.example.java.graph.ConnectedComponents; -import eu.stratosphere.example.java.record.kmeans.KMeans; -import eu.stratosphere.example.java.record.relational.TPCHQuery3; -import eu.stratosphere.example.java.record.relational.WebLogAnalysis; -import eu.stratosphere.example.java.record.wordcount.WordCount; import eu.stratosphere.test.compiler.CompilerTestBase; -import eu.stratosphere.test.testPrograms.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; +import eu.stratosphere.test.recordJobs.relational.WebLogAnalysis; +import eu.stratosphere.test.recordJobs.wordcount.WordCount; /* * The tests in this class simply invokes the JSON dump code for the optimized plan. diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java index d5044e8867d58..f1cc11ac40791 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java @@ -28,11 +28,11 @@ import eu.stratosphere.compiler.dag.DataSinkNode; import eu.stratosphere.compiler.plandump.PlanJSONDumpGenerator; import eu.stratosphere.example.java.graph.ConnectedComponents; -import eu.stratosphere.example.java.record.kmeans.KMeans; -import eu.stratosphere.example.java.record.relational.TPCHQuery3; -import eu.stratosphere.example.java.record.relational.WebLogAnalysis; -import eu.stratosphere.example.java.record.wordcount.WordCount; -import eu.stratosphere.test.testPrograms.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; +import eu.stratosphere.test.recordJobs.relational.WebLogAnalysis; +import eu.stratosphere.test.recordJobs.wordcount.WordCount; import eu.stratosphere.util.OperatingSystem; /* diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java index e6ff269bea467..d2caeb7063a91 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java @@ -17,7 +17,7 @@ import eu.stratosphere.configuration.Configuration; import eu.stratosphere.examples.scala.graph.ComputeEdgeDegrees; -public class ComputeEdgeDegreesITCase extends eu.stratosphere.test.exampleRecordPrograms.ComputeEdgeDegreesITCase { +public class ComputeEdgeDegreesITCase extends eu.stratosphere.test.recordJobTests.ComputeEdgeDegreesITCase { public ComputeEdgeDegreesITCase(Configuration config) { super(config); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java index c188d78a8685d..5801d5980cd8c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java @@ -17,7 +17,7 @@ import eu.stratosphere.configuration.Configuration; import eu.stratosphere.examples.scala.graph.EnumTrianglesOnEdgesWithDegrees; -public class EnumTrianglesOnEdgesWithDegreesITCase extends eu.stratosphere.test.exampleRecordPrograms.EnumTrianglesOnEdgesWithDegreesITCase { +public class EnumTrianglesOnEdgesWithDegreesITCase extends eu.stratosphere.test.recordJobTests.EnumTrianglesOnEdgesWithDegreesITCase { public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) { super(config); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/RelationalQueryITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/RelationalQueryITCase.java index da5b18abf3c97..3e144a3394fd8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/RelationalQueryITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/RelationalQueryITCase.java @@ -23,7 +23,7 @@ import java.util.Locale; @RunWith(Parameterized.class) -public class RelationalQueryITCase extends eu.stratosphere.test.exampleRecordPrograms.TPCHQuery3ITCase { +public class RelationalQueryITCase extends eu.stratosphere.test.recordJobTests.TPCHQuery3ITCase { public RelationalQueryITCase(Configuration config) { super(config); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java index e33eeea8c19b7..2ddef0c0fcd35 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WebLogAnalysisITCase.java @@ -16,7 +16,7 @@ import eu.stratosphere.api.common.Plan; import eu.stratosphere.examples.scala.relational.WebLogAnalysis; -public class WebLogAnalysisITCase extends eu.stratosphere.test.exampleRecordPrograms.WebLogAnalysisITCase { +public class WebLogAnalysisITCase extends eu.stratosphere.test.recordJobTests.WebLogAnalysisITCase { @Override protected Plan getTestJob() { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java index 005d4760271a2..205828ff70ff0 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountITCase.java @@ -17,7 +17,7 @@ import eu.stratosphere.examples.scala.wordcount.WordCount; -public class WordCountITCase extends eu.stratosphere.test.exampleRecordPrograms.WordCountITCase { +public class WordCountITCase extends eu.stratosphere.test.recordJobTests.WordCountITCase { @Override protected Plan getTestJob() { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java index 6f9508eb14f95..4627d487845ab 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountPactValueITCase.java @@ -17,7 +17,7 @@ import eu.stratosphere.examples.scala.wordcount.WordCountWithUserDefinedType; -public class WordCountPactValueITCase extends eu.stratosphere.test.exampleRecordPrograms.WordCountITCase { +public class WordCountPactValueITCase extends eu.stratosphere.test.recordJobTests.WordCountITCase { @Override protected Plan getTestJob() { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java index 982b6f5c7bd10..12082ac33bde8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java @@ -16,7 +16,7 @@ import eu.stratosphere.api.common.Plan; import eu.stratosphere.examples.scala.wordcount.WordCountWithCount; -public class WordCountWithCountFunctionITCase extends eu.stratosphere.test.exampleRecordPrograms.WordCountITCase { +public class WordCountWithCountFunctionITCase extends eu.stratosphere.test.recordJobTests.WordCountITCase { @Override protected Plan getTestJob() { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java index 76e45731b8151..50c9fad48d87b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java @@ -30,8 +30,8 @@ import eu.stratosphere.api.java.record.operators.CoGroupOperator.CombinableFirst; import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.DuplicateLongMap; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; import eu.stratosphere.test.testdata.ConnectedComponentsData; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java index b1e68d33e15da..9c88bb5c5d985 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsITCase.java @@ -16,7 +16,7 @@ import java.io.BufferedReader; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents; import eu.stratosphere.test.testdata.ConnectedComponentsData; import eu.stratosphere.test.util.RecordAPITestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java index 04b4b1c0dc363..a5ad1fe369ab1 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java @@ -34,9 +34,9 @@ import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.DuplicateLongMap; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.MinimumComponentIDReduce; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; import eu.stratosphere.test.testdata.ConnectedComponentsData; import eu.stratosphere.types.LongValue; import eu.stratosphere.types.Record; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java index 126bffb36c5fc..f97b389acf8d9 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java @@ -27,9 +27,9 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.DuplicateLongMap; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.MinimumComponentIDReduce; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; import eu.stratosphere.test.testdata.ConnectedComponentsData; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java index fe2f3f2235e58..e84f21e7582f1 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DanglingPageRankITCase.java @@ -15,15 +15,15 @@ import java.util.Collection; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.pagerank.DanglingPageRank; import eu.stratosphere.test.iterative.nephele.DanglingPageRankNepheleITCase; +import eu.stratosphere.test.recordJobs.graph.DanglingPageRank; +import eu.stratosphere.test.util.RecordAPITestBase; @RunWith(Parameterized.class) public class DanglingPageRankITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java index 1c966a402ef4c..8f069294351dd 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/DeltaPageRankITCase.java @@ -22,7 +22,7 @@ import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents; @RunWith(Parameterized.class) public class DeltaPageRankITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java index c5aed9a82204c..9cbb0cdd7dc99 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithChainingITCase.java @@ -12,6 +12,14 @@ **********************************************************************************************************************/ package eu.stratosphere.test.iterative; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.operators.BulkIteration; import eu.stratosphere.api.common.operators.FileDataSink; @@ -21,21 +29,13 @@ import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.kmeans.udfs.PointInFormat; -import eu.stratosphere.example.java.record.kmeans.udfs.PointOutFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointInFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointOutFormat; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; - @RunWith(Parameterized.class) public class IterationWithChainingITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java index a704536e1cd51..90e4b39da05d8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterationWithUnionITCase.java @@ -17,7 +17,6 @@ import java.util.Collection; import java.util.Iterator; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -30,8 +29,9 @@ import eu.stratosphere.api.java.record.functions.ReduceFunction; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.kmeans.udfs.PointInFormat; -import eu.stratosphere.example.java.record.kmeans.udfs.PointOutFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointOutFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointInFormat; +import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java index b81b1953d15e8..d8f46aee7b5bb 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java @@ -17,7 +17,7 @@ import java.util.List; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.example.java.record.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeans; import eu.stratosphere.test.testdata.KMeansData; import eu.stratosphere.test.util.RecordAPITestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java index 944869f7c9536..78e8908a59ab4 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java @@ -17,7 +17,7 @@ import java.util.List; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.example.java.record.kmeans.KMeansTutorialExample; +import eu.stratosphere.test.recordJobs.kmeans.KMeansTutorialExample; import eu.stratosphere.test.testdata.KMeansData; import eu.stratosphere.test.util.RecordAPITestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java index 5e4889114a748..8b12b53932339 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/PageRankITCase.java @@ -15,14 +15,14 @@ import java.util.Collection; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.pagerank.SimplePageRank; +import eu.stratosphere.test.recordJobs.graph.SimplePageRank; +import eu.stratosphere.test.util.RecordAPITestBase; @RunWith(Parameterized.class) public class PageRankITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java index 443943b33fca5..c4ebcca901688 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java @@ -54,9 +54,9 @@ import eu.stratosphere.pact.runtime.task.chaining.ChainedCollectorMapDriver; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.pact.runtime.task.util.TaskConfig; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.MinimumComponentIDReduce; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.NeighborWithComponentIDJoin; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.UpdateComponentIdMatch; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.UpdateComponentIdMatch; import eu.stratosphere.test.testdata.ConnectedComponentsData; import eu.stratosphere.types.LongValue; import eu.stratosphere.types.Record; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java index a7ef31dbabd8f..8b38228283a95 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java @@ -12,6 +12,12 @@ **********************************************************************************************************************/ package eu.stratosphere.test.iterative.nephele; +import java.util.Collection; +import java.util.Iterator; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import eu.stratosphere.api.common.operators.util.UserCodeClassWrapper; import eu.stratosphere.api.common.typeutils.TypeComparatorFactory; import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; @@ -19,34 +25,32 @@ import eu.stratosphere.api.java.record.functions.ReduceFunction; import eu.stratosphere.api.java.record.io.FileOutputFormat; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.kmeans.udfs.CoordVector; -import eu.stratosphere.example.java.record.kmeans.udfs.PointInFormat; -import eu.stratosphere.example.java.record.kmeans.udfs.PointOutFormat; import eu.stratosphere.nephele.io.DistributionPattern; import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.jobgraph.*; +import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException; +import eu.stratosphere.nephele.jobgraph.JobInputVertex; +import eu.stratosphere.nephele.jobgraph.JobOutputVertex; +import eu.stratosphere.nephele.jobgraph.JobTaskVertex; import eu.stratosphere.pact.runtime.iterative.task.IterationHeadPactTask; import eu.stratosphere.pact.runtime.iterative.task.IterationTailPactTask; import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparatorFactory; import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordSerializerFactory; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; -import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.CollectorMapDriver; +import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.GroupReduceDriver; import eu.stratosphere.pact.runtime.task.chaining.ChainedCollectorMapDriver; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.pact.runtime.task.util.TaskConfig; +import eu.stratosphere.test.recordJobs.kmeans.udfs.CoordVector; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointInFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointOutFormat; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.Iterator; - /** * Tests chained iteration tails. *

diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/CollectionSourceTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/CollectionSourceTest.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java index 2102b7b4dbc67..6f0aaaeced926 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/CollectionSourceTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionSourceTest.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.io.Serializable; import java.util.ArrayList; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/CollectionValidationTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionValidationTest.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/CollectionValidationTest.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionValidationTest.java index e53c561c6f0d2..7a380f14cd131 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/CollectionValidationTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/CollectionValidationTest.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.java.record.operators.CollectionDataSource; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/ComputeEdgeDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/ComputeEdgeDegreesITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java index 162d0c2e35ec6..f52e1086d63f3 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/ComputeEdgeDegreesITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/ComputeEdgeDegreesITCase.java @@ -11,18 +11,18 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.util.Collection; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.triangles.ComputeEdgeDegrees; +import eu.stratosphere.test.recordJobs.graph.ComputeEdgeDegrees; +import eu.stratosphere.test.util.RecordAPITestBase; @RunWith(Parameterized.class) public class ComputeEdgeDegreesITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java similarity index 94% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java index 7bb8c065ca745..b4b0386d3fd61 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java @@ -11,18 +11,18 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.util.Collection; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.triangles.EnumTrianglesOnEdgesWithDegrees; +import eu.stratosphere.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees; +import eu.stratosphere.test.util.RecordAPITestBase; @RunWith(Parameterized.class) public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/EnumTrianglesRDFITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/EnumTrianglesRDFITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java index 5ba8beb2cb9f5..945cc67f3084c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/EnumTrianglesRDFITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/EnumTrianglesRDFITCase.java @@ -11,11 +11,11 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.triangles.EnumTrianglesRdfFoaf; +import eu.stratosphere.test.recordJobs.graph.EnumTrianglesRdfFoaf; import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GlobalSortingITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GlobalSortingITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java index ac989cae40c62..69b3776513788 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GlobalSortingITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingITCase.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.util.ArrayList; import java.util.Collections; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GlobalSortingMixedOrderITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GlobalSortingMixedOrderITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java index 310854bdaca85..e7d47702290ce 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GlobalSortingMixedOrderITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GroupOrderReduceITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GroupOrderReduceITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java index cfbc5c7c8788f..5e0d439544f42 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/GroupOrderReduceITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GroupOrderReduceITCase.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.io.Serializable; import java.util.Collection; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/KMeansStepITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/KMeansStepITCase.java similarity index 94% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/KMeansStepITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/KMeansStepITCase.java index a66240f942d2c..195d8a7ccbce2 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/KMeansStepITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/KMeansStepITCase.java @@ -11,13 +11,13 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.util.ArrayList; import java.util.List; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.testPrograms.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; import eu.stratosphere.test.testdata.KMeansData; import eu.stratosphere.test.util.RecordAPITestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/MergeOnlyJoinITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/MergeOnlyJoinITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java index d7607e19b3b9c..47a0f59256007 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/MergeOnlyJoinITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/MergeOnlyJoinITCase.java @@ -11,11 +11,11 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.mergeOnlyJoin.MergeOnlyJoin; +import eu.stratosphere.test.recordJobs.relational.MergeOnlyJoin; import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/PairwiseSPITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java similarity index 97% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/PairwiseSPITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java index b44b80f436393..38168a2b602e0 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/PairwiseSPITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/PairwiseSPITCase.java @@ -11,17 +11,18 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; + +import java.util.Collection; -import eu.stratosphere.api.common.Plan; -import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.shortestpaths.PairwiseSP; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import java.util.Collection; +import eu.stratosphere.api.common.Plan; +import eu.stratosphere.configuration.Configuration; +import eu.stratosphere.test.recordJobs.graph.PairwiseSP; +import eu.stratosphere.test.util.RecordAPITestBase; @RunWith(Parameterized.class) public class PairwiseSPITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery10ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery10ITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java index 4d0980efd86d3..a19cad1e1efd5 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery10ITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery10ITCase.java @@ -11,11 +11,11 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.tpch10.TPCHQuery10; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery10; import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery3ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery3ITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java index cb7a2a9b238b4..cc1d16a9f6dca 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery3ITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3ITCase.java @@ -11,18 +11,18 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.util.Collection; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import eu.stratosphere.api.common.Plan; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.relational.TPCHQuery3; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; +import eu.stratosphere.test.util.RecordAPITestBase; @RunWith(Parameterized.class) public class TPCHQuery3ITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery3WithUnionITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery3WithUnionITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java index 33f9e4a61e416..b0c52003f5f2d 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery3WithUnionITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery3WithUnionITCase.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.testPrograms.tpch3Unioned.TPCHQuery3Unioned; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3Unioned; import eu.stratosphere.test.util.RecordAPITestBase; public class TPCHQuery3WithUnionITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery4ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery4ITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java index d12a7e7789dc6..d8b92751928cc 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery4ITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery4ITCase.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.testPrograms.tpch4.TPCHQuery4; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery4; import eu.stratosphere.test.util.RecordAPITestBase; public class TPCHQuery4ITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery9ITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery9ITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java index 36a3ef3a2ef1a..b1e9af6fe17f7 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQuery9ITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQuery9ITCase.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.testPrograms.tpch9.TPCHQuery9; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery9; import eu.stratosphere.test.util.RecordAPITestBase; public class TPCHQuery9ITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQueryAsterixITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQueryAsterixITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java index 7187fc49a1a8e..9c2ba2662acd5 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TPCHQueryAsterixITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TPCHQueryAsterixITCase.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.testPrograms.TPCHQueryAsterix; +import eu.stratosphere.test.recordJobs.relational.TPCHQueryAsterix; import eu.stratosphere.test.util.RecordAPITestBase; public class TPCHQueryAsterixITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TeraSortITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TeraSortITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java index b0c583a03f7b7..97db904c9d79f 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/TeraSortITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/TeraSortITCase.java @@ -11,17 +11,17 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.io.File; import java.io.FileInputStream; import java.net.URI; -import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.Assert; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.example.java.record.sort.TeraSort; +import eu.stratosphere.test.recordJobs.sort.TeraSort; +import eu.stratosphere.test.util.RecordAPITestBase; public class TeraSortITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WebLogAnalysisITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WebLogAnalysisITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java index a6b7d0cac36af..8b60d52e722f8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WebLogAnalysisITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WebLogAnalysisITCase.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.example.java.record.relational.WebLogAnalysis; +import eu.stratosphere.test.recordJobs.relational.WebLogAnalysis; import eu.stratosphere.test.util.RecordAPITestBase; public class WebLogAnalysisITCase extends RecordAPITestBase { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java similarity index 93% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WordCountITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java index 943582376099d..1a2d183457da4 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WordCountITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountITCase.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.example.java.record.wordcount.WordCount; +import eu.stratosphere.test.recordJobs.wordcount.WordCount; import eu.stratosphere.test.testdata.WordCountData; import eu.stratosphere.test.util.RecordAPITestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WordCountUnionReduceITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WordCountUnionReduceITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java index cdf601405b5f2..72abc349871ca 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleRecordPrograms/WordCountUnionReduceITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.test.exampleRecordPrograms; +package eu.stratosphere.test.recordJobTests; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -22,10 +22,10 @@ import eu.stratosphere.api.java.record.io.TextInputFormat; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.example.java.record.wordcount.WordCount.CountWords; -import eu.stratosphere.example.java.record.wordcount.WordCount.TokenizeLine; import eu.stratosphere.nephele.io.MutableUnionRecordReader; import eu.stratosphere.nephele.io.UnionRecordReader; +import eu.stratosphere.test.recordJobs.wordcount.WordCount.CountWords; +import eu.stratosphere.test.recordJobs.wordcount.WordCount.TokenizeLine; import eu.stratosphere.test.testdata.WordCountData; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.IntValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/ComputeEdgeDegrees.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ComputeEdgeDegrees.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/ComputeEdgeDegrees.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ComputeEdgeDegrees.java index 7a17f1da4199b..65f19e4940c2c 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/ComputeEdgeDegrees.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ComputeEdgeDegrees.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles; +package eu.stratosphere.test.recordJobs.graph; import java.io.Serializable; import java.util.Arrays; @@ -26,8 +26,8 @@ import eu.stratosphere.api.java.record.functions.ReduceFunction; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.example.java.record.triangles.io.EdgeInputFormat; -import eu.stratosphere.example.java.record.triangles.io.EdgeWithDegreesOutputFormat; +import eu.stratosphere.test.recordJobs.graph.triangleEnumUtil.EdgeInputFormat; +import eu.stratosphere.test.recordJobs.graph.triangleEnumUtil.EdgeWithDegreesOutputFormat; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/ConnectedComponentsWithCoGroup.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/ConnectedComponentsWithCoGroup.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java index a1e6f532d65fc..923e4ad50f7f8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/ConnectedComponentsWithCoGroup.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms; +package eu.stratosphere.test.recordJobs.graph; import java.io.Serializable; import java.util.Iterator; @@ -30,8 +30,8 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.CoGroupOperator.CombinableFirst; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.DuplicateLongMap; -import eu.stratosphere.test.testPrograms.WorksetConnectedComponents.NeighborWithComponentIDJoin; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; +import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; import eu.stratosphere.types.LongValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DanglingPageRank.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/DanglingPageRank.java similarity index 86% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DanglingPageRank.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/DanglingPageRank.java index 8c9f176f12143..f6656a6923704 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DanglingPageRank.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/DanglingPageRank.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; @@ -21,6 +21,13 @@ import eu.stratosphere.api.common.operators.FileDataSource; import eu.stratosphere.api.java.record.operators.CoGroupOperator; import eu.stratosphere.api.java.record.operators.JoinOperator; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.DanglingPageRankInputFormat; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.DiffL1NormConvergenceCriterion; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.DotProductCoGroup; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.DotProductMatch; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.ImprovedAdjacencyListInputFormat; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.PageRankStatsAggregator; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.PageWithRankOutFormat; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/incremental/pagerank/DeltaPageRankWithInitialDeltas.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/incremental/pagerank/DeltaPageRankWithInitialDeltas.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java index 35651f06f0c2d..814d10be2f71d 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/incremental/pagerank/DeltaPageRankWithInitialDeltas.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/DeltaPageRankWithInitialDeltas.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.incremental.pagerank; +package eu.stratosphere.test.recordJobs.graph; import java.util.Iterator; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesOnEdgesWithDegrees.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java similarity index 96% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesOnEdgesWithDegrees.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java index a56e81e791ee6..e705147829c28 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesOnEdgesWithDegrees.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesOnEdgesWithDegrees.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles; +package eu.stratosphere.test.recordJobs.graph; import java.io.Serializable; import java.util.Iterator; @@ -27,8 +27,8 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.example.java.record.triangles.io.EdgeWithDegreesInputFormat; -import eu.stratosphere.example.java.record.triangles.io.TriangleOutputFormat; +import eu.stratosphere.test.recordJobs.graph.triangleEnumUtil.EdgeWithDegreesInputFormat; +import eu.stratosphere.test.recordJobs.graph.triangleEnumUtil.TriangleOutputFormat; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesRdfFoaf.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesRdfFoaf.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesRdfFoaf.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesRdfFoaf.java index ed39ad6af0e1c..6bc5d7e6a8743 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesRdfFoaf.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesRdfFoaf.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles; +package eu.stratosphere.test.recordJobs.graph; import java.io.Serializable; import java.util.ArrayList; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesWithDegrees.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesWithDegrees.java similarity index 82% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesWithDegrees.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesWithDegrees.java index 32147e342169c..152cb529b9573 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/EnumTrianglesWithDegrees.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/EnumTrianglesWithDegrees.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles; +package eu.stratosphere.test.recordJobs.graph; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; @@ -21,15 +21,15 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.example.java.record.triangles.ComputeEdgeDegrees.CountEdges; -import eu.stratosphere.example.java.record.triangles.ComputeEdgeDegrees.JoinCountsAndUniquify; -import eu.stratosphere.example.java.record.triangles.ComputeEdgeDegrees.ProjectEdge; -import eu.stratosphere.example.java.record.triangles.EnumTrianglesOnEdgesWithDegrees.BuildTriads; -import eu.stratosphere.example.java.record.triangles.EnumTrianglesOnEdgesWithDegrees.CloseTriads; -import eu.stratosphere.example.java.record.triangles.EnumTrianglesOnEdgesWithDegrees.ProjectOutCounts; -import eu.stratosphere.example.java.record.triangles.EnumTrianglesOnEdgesWithDegrees.ProjectToLowerDegreeVertex; -import eu.stratosphere.example.java.record.triangles.io.EdgeInputFormat; -import eu.stratosphere.example.java.record.triangles.io.TriangleOutputFormat; +import eu.stratosphere.test.recordJobs.graph.ComputeEdgeDegrees.CountEdges; +import eu.stratosphere.test.recordJobs.graph.ComputeEdgeDegrees.JoinCountsAndUniquify; +import eu.stratosphere.test.recordJobs.graph.ComputeEdgeDegrees.ProjectEdge; +import eu.stratosphere.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.BuildTriads; +import eu.stratosphere.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.CloseTriads; +import eu.stratosphere.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.ProjectOutCounts; +import eu.stratosphere.test.recordJobs.graph.EnumTrianglesOnEdgesWithDegrees.ProjectToLowerDegreeVertex; +import eu.stratosphere.test.recordJobs.graph.triangleEnumUtil.EdgeInputFormat; +import eu.stratosphere.test.recordJobs.graph.triangleEnumUtil.TriangleOutputFormat; import eu.stratosphere.types.IntValue; /** diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/shortestpaths/PairwiseSP.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/PairwiseSP.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/shortestpaths/PairwiseSP.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/PairwiseSP.java index 3fa128c36cd9e..b0e3236142d47 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/shortestpaths/PairwiseSP.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/PairwiseSP.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.shortestpaths; +package eu.stratosphere.test.recordJobs.graph; import java.io.IOException; import java.io.Serializable; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/SimplePageRank.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/SimplePageRank.java similarity index 94% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/SimplePageRank.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/SimplePageRank.java index 7fdfe71b4bad0..46748dfb6371e 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/SimplePageRank.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/SimplePageRank.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph; import java.io.Serializable; import java.util.Iterator; @@ -28,6 +28,10 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.DanglingPageRankInputFormat; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.ImprovedAdjacencyListInputFormat; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.LongArrayView; +import eu.stratosphere.test.recordJobs.graph.pageRankUtil.PageWithRankOutFormat; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/WorksetConnectedComponents.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/WorksetConnectedComponents.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/WorksetConnectedComponents.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/WorksetConnectedComponents.java index d695c3106e9ec..a15764a6e27ea 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/WorksetConnectedComponents.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/WorksetConnectedComponents.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms; +package eu.stratosphere.test.recordJobs.graph; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/AsciiLongArrayView.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/AsciiLongArrayView.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java index 7f36c476c8780..3837e648dfd46 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/AsciiLongArrayView.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import java.io.Serializable; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DanglingPageRankInputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.java similarity index 95% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DanglingPageRankInputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.java index e1304a9f10258..3481ee4744fc5 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DanglingPageRankInputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.java @@ -11,11 +11,11 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import eu.stratosphere.api.java.record.io.TextInputFormat; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.util.ConfigUtils; +import eu.stratosphere.test.recordJobs.util.ConfigUtils; import eu.stratosphere.types.BooleanValue; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DiffL1NormConvergenceCriterion.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java similarity index 96% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DiffL1NormConvergenceCriterion.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java index 3e80d3276d558..cc7b06a612ade 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DiffL1NormConvergenceCriterion.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DotProductCoGroup.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DotProductCoGroup.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java index f2d3f6dd82e80..74ba01ea7e3e2 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DotProductCoGroup.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import java.io.Serializable; import java.util.Iterator; @@ -19,7 +19,7 @@ import eu.stratosphere.api.java.record.functions.CoGroupFunction; import eu.stratosphere.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.record.util.ConfigUtils; +import eu.stratosphere.test.recordJobs.util.ConfigUtils; import eu.stratosphere.types.BooleanValue; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DotProductMatch.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DotProductMatch.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DotProductMatch.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DotProductMatch.java index f0f715b3489fa..f94d67140b779 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/DotProductMatch.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/DotProductMatch.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import java.io.Serializable; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/ImprovedAdjacencyListInputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/ImprovedAdjacencyListInputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java index e7e1895e89609..59b38b7e4271c 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/ImprovedAdjacencyListInputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import eu.stratosphere.api.java.record.io.TextInputFormat; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/LongArrayView.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/LongArrayView.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java index f6042830a4d19..317d5fceb7be2 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/LongArrayView.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageRankStats.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageRankStats.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java index 9b2850c5aa5b6..20558ed175b5f 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageRankStats.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageRankStatsAggregator.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageRankStatsAggregator.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java index ba2d579b0b929..569f2e03c72f7 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageRankStatsAggregator.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import eu.stratosphere.api.common.aggregators.Aggregator; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageWithRankOutFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java similarity index 96% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageWithRankOutFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java index 15ed4dd4a98de..0eba47e06de35 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/pagerank/PageWithRankOutFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.pagerank; +package eu.stratosphere.test.recordJobs.graph.pageRankUtil; import eu.stratosphere.api.java.record.io.DelimitedOutputFormat; import eu.stratosphere.types.DoubleValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeInputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeInputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java index 425c0ac92872c..db4bfcd91a2cc 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeInputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeInputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles.io; +package eu.stratosphere.test.recordJobs.graph.triangleEnumUtil; import eu.stratosphere.api.java.record.io.DelimitedInputFormat; import eu.stratosphere.configuration.Configuration; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeWithDegreesInputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeWithDegreesInputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java index 89869a163e552..ab2c47da59fb3 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeWithDegreesInputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesInputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles.io; +package eu.stratosphere.test.recordJobs.graph.triangleEnumUtil; import eu.stratosphere.api.java.record.io.DelimitedInputFormat; import eu.stratosphere.configuration.Configuration; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeWithDegreesOutputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeWithDegreesOutputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java index 58d12cfdfad0f..fd06601c24713 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/EdgeWithDegreesOutputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/EdgeWithDegreesOutputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles.io; +package eu.stratosphere.test.recordJobs.graph.triangleEnumUtil; import eu.stratosphere.api.java.record.io.DelimitedOutputFormat; import eu.stratosphere.types.IntValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/TriangleOutputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java similarity index 96% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/TriangleOutputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java index c82d74a42a3aa..56cae0b0feab7 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/triangles/io/TriangleOutputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/triangleEnumUtil/TriangleOutputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.triangles.io; +package eu.stratosphere.test.recordJobs.graph.triangleEnumUtil; import eu.stratosphere.api.java.record.io.DelimitedOutputFormat; import eu.stratosphere.types.IntValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeans.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeans.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeans.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeans.java index f35b9768792a9..cca940aa50dac 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeans.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeans.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.kmeans; +package eu.stratosphere.test.recordJobs.kmeans; import java.io.DataInput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansIterative.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansIterative.java similarity index 92% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansIterative.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansIterative.java index ae7d28c0a152e..4451e77feae19 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansIterative.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansIterative.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.kmeans; +package eu.stratosphere.test.recordJobs.kmeans; import java.util.ArrayList; import java.util.List; @@ -26,11 +26,11 @@ import eu.stratosphere.api.java.record.operators.CrossOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.client.LocalExecutor; -import eu.stratosphere.example.java.record.kmeans.udfs.ComputeDistance; -import eu.stratosphere.example.java.record.kmeans.udfs.FindNearestCenter; -import eu.stratosphere.example.java.record.kmeans.udfs.PointInFormat; -import eu.stratosphere.example.java.record.kmeans.udfs.PointOutFormat; -import eu.stratosphere.example.java.record.kmeans.udfs.RecomputeClusterCenter; +import eu.stratosphere.test.recordJobs.kmeans.udfs.ComputeDistance; +import eu.stratosphere.test.recordJobs.kmeans.udfs.FindNearestCenter; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointInFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.PointOutFormat; +import eu.stratosphere.test.recordJobs.kmeans.udfs.RecomputeClusterCenter; import eu.stratosphere.types.IntValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/KMeansSingleStep.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java similarity index 91% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/KMeansSingleStep.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java index 1ad29d6089349..500b6be221b44 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/KMeansSingleStep.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms; +package eu.stratosphere.test.recordJobs.kmeans; import eu.stratosphere.api.common.Plan; @@ -22,10 +22,10 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.example.java.record.kmeans.KMeans.PointBuilder; -import eu.stratosphere.example.java.record.kmeans.KMeans.PointOutFormat; -import eu.stratosphere.example.java.record.kmeans.KMeans.RecomputeClusterCenter; -import eu.stratosphere.example.java.record.kmeans.KMeans.SelectNearestCenter; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointBuilder; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointOutFormat; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.RecomputeClusterCenter; +import eu.stratosphere.test.recordJobs.kmeans.KMeans.SelectNearestCenter; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansTutorialExample.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansTutorialExample.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java index 2e42e36ed687d..dd3155b2f35d1 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/KMeansTutorialExample.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.kmeans; +package eu.stratosphere.test.recordJobs.kmeans; import java.io.DataInput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/ComputeDistance.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/ComputeDistance.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/ComputeDistance.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/ComputeDistance.java index 0e0b1eae36914..37aa16f217b71 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/ComputeDistance.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/ComputeDistance.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.io.Serializable; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/ComputeDistanceParameterized.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/ComputeDistanceParameterized.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java index 739031d123f90..ce4372e27b12f 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/ComputeDistanceParameterized.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/ComputeDistanceParameterized.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.io.Serializable; import java.util.Collection; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/CoordVector.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/CoordVector.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java index cfd8cf5fe4f8b..e2b0420eb2796 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/CoordVector.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/FindNearestCenter.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/FindNearestCenter.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/FindNearestCenter.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/FindNearestCenter.java index 3ffe2967b5752..ecec722c025aa 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/FindNearestCenter.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/FindNearestCenter.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/PointInFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/PointInFormat.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/PointInFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/PointInFormat.java index d9931ec816edd..ce9619743907c 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/PointInFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/PointInFormat.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.util.ArrayList; import java.util.List; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/PointOutFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/PointOutFormat.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/PointOutFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/PointOutFormat.java index b03b32525c064..5560ee1dde430 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/PointOutFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/PointOutFormat.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.text.DecimalFormat; import java.text.DecimalFormatSymbols; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/RecomputeClusterCenter.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/RecomputeClusterCenter.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java index 3fdaa4a783833..7aefa5130d86a 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/kmeans/udfs/RecomputeClusterCenter.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/RecomputeClusterCenter.java @@ -10,7 +10,7 @@ * 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 eu.stratosphere.example.java.record.kmeans.udfs; +package eu.stratosphere.test.recordJobs.kmeans.udfs; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/mergeOnlyJoin/MergeOnlyJoin.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/MergeOnlyJoin.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/mergeOnlyJoin/MergeOnlyJoin.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/MergeOnlyJoin.java index 528269c11b730..194a9bfdd7a0b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/mergeOnlyJoin/MergeOnlyJoin.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/MergeOnlyJoin.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.mergeOnlyJoin; +package eu.stratosphere.test.recordJobs.relational; import java.util.Iterator; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/TPCHQuery1.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery1.java similarity index 89% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/TPCHQuery1.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery1.java index 3f7d558f04ff0..132bd3041ba71 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/TPCHQuery1.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery1.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch1; +package eu.stratosphere.test.recordJobs.relational; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; @@ -20,8 +20,10 @@ import eu.stratosphere.api.common.operators.FileDataSource; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.test.testPrograms.util.IntTupleDataInFormat; -import eu.stratosphere.test.testPrograms.util.StringTupleDataOutFormat; +import eu.stratosphere.test.recordJobs.relational.query1Util.GroupByReturnFlag; +import eu.stratosphere.test.recordJobs.relational.query1Util.LineItemFilter; +import eu.stratosphere.test.recordJobs.util.IntTupleDataInFormat; +import eu.stratosphere.test.recordJobs.util.StringTupleDataOutFormat; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch10/TPCHQuery10.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery10.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch10/TPCHQuery10.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery10.java index 60f07920f0f69..305c274f480db 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch10/TPCHQuery10.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery10.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch10; +package eu.stratosphere.test.recordJobs.relational; import java.io.IOException; import java.text.DecimalFormat; @@ -30,8 +30,8 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.test.testPrograms.util.IntTupleDataInFormat; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.IntTupleDataInFormat; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/TPCHQuery3.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery3.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/TPCHQuery3.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery3.java index 2c085103fb09b..1e984ed8be7fc 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/TPCHQuery3.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery3.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.relational; +package eu.stratosphere.test.recordJobs.relational; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch3Unioned/TPCHQuery3Unioned.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery3Unioned.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch3Unioned/TPCHQuery3Unioned.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery3Unioned.java index 600bcbd57f836..6db441e49d38f 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch3Unioned/TPCHQuery3Unioned.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery3Unioned.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch3Unioned; +package eu.stratosphere.test.recordJobs.relational; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; @@ -23,10 +23,9 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.example.java.record.relational.TPCHQuery3; -import eu.stratosphere.example.java.record.relational.TPCHQuery3.AggLiO; -import eu.stratosphere.example.java.record.relational.TPCHQuery3.FilterO; -import eu.stratosphere.example.java.record.relational.TPCHQuery3.JoinLiO; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3.AggLiO; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3.FilterO; +import eu.stratosphere.test.recordJobs.relational.TPCHQuery3.JoinLiO; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.LongValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch4/TPCHQuery4.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery4.java similarity index 97% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch4/TPCHQuery4.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery4.java index 8c69e0c165355..9907b05f7d49b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch4/TPCHQuery4.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery4.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch4; +package eu.stratosphere.test.recordJobs.relational; import java.text.ParseException; import java.text.SimpleDateFormat; @@ -34,9 +34,9 @@ import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.test.testPrograms.util.IntTupleDataInFormat; -import eu.stratosphere.test.testPrograms.util.StringTupleDataOutFormat; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.IntTupleDataInFormat; +import eu.stratosphere.test.recordJobs.util.StringTupleDataOutFormat; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/TPCHQuery9.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery9.java similarity index 88% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/TPCHQuery9.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery9.java index 35637b29b4600..0fb2627a8e93b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/TPCHQuery9.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQuery9.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational; import org.apache.log4j.Logger; @@ -23,7 +23,21 @@ import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.test.testPrograms.util.IntTupleDataInFormat; +import eu.stratosphere.test.recordJobs.relational.query9Util.AmountAggregate; +import eu.stratosphere.test.recordJobs.relational.query9Util.FilteredPartsJoin; +import eu.stratosphere.test.recordJobs.relational.query9Util.IntPair; +import eu.stratosphere.test.recordJobs.relational.query9Util.LineItemMap; +import eu.stratosphere.test.recordJobs.relational.query9Util.OrderMap; +import eu.stratosphere.test.recordJobs.relational.query9Util.OrderedPartsJoin; +import eu.stratosphere.test.recordJobs.relational.query9Util.PartFilter; +import eu.stratosphere.test.recordJobs.relational.query9Util.PartJoin; +import eu.stratosphere.test.recordJobs.relational.query9Util.PartListJoin; +import eu.stratosphere.test.recordJobs.relational.query9Util.PartsuppMap; +import eu.stratosphere.test.recordJobs.relational.query9Util.StringIntPair; +import eu.stratosphere.test.recordJobs.relational.query9Util.StringIntPairStringDataOutFormat; +import eu.stratosphere.test.recordJobs.relational.query9Util.SupplierMap; +import eu.stratosphere.test.recordJobs.relational.query9Util.SuppliersJoin; +import eu.stratosphere.test.recordJobs.util.IntTupleDataInFormat; import eu.stratosphere.types.IntValue; /** diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/TPCHQueryAsterix.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQueryAsterix.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/TPCHQueryAsterix.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQueryAsterix.java index a2b95ef02875c..126fcee52be46 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/TPCHQueryAsterix.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/TPCHQueryAsterix.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms; +package eu.stratosphere.test.recordJobs.relational; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/WebLogAnalysis.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/WebLogAnalysis.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/WebLogAnalysis.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/WebLogAnalysis.java index 2ea61e5e61ba3..e1b49cd0b64bc 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/relational/WebLogAnalysis.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/WebLogAnalysis.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.relational; +package eu.stratosphere.test.recordJobs.relational; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/GroupByReturnFlag.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/GroupByReturnFlag.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/GroupByReturnFlag.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/GroupByReturnFlag.java index d49734dbaf51f..06136b2c759e3 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/GroupByReturnFlag.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/GroupByReturnFlag.java @@ -11,12 +11,12 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch1; +package eu.stratosphere.test.recordJobs.relational.query1Util; import java.util.Iterator; import eu.stratosphere.api.java.record.functions.ReduceFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.LongValue; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/LineItemFilter.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilter.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/LineItemFilter.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilter.java index 7120d72edf621..64b3a8c252323 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/LineItemFilter.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilter.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch1; +package eu.stratosphere.test.recordJobs.relational.query1Util; import java.text.DateFormat; import java.text.ParseException; @@ -21,7 +21,7 @@ import org.apache.log4j.Logger; import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/LineItemFilterTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/LineItemFilterTest.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java index 008fc12cbe1cb..03e442ee645e6 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch1/LineItemFilterTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch1; +package eu.stratosphere.test.recordJobs.relational.query1Util; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.verify; @@ -28,7 +28,7 @@ import eu.stratosphere.nephele.io.AbstractRecordWriter; import eu.stratosphere.pact.runtime.shipping.RecordOutputCollector; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/AmountAggregate.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/AmountAggregate.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/AmountAggregate.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/AmountAggregate.java index 4ca6e0a06e406..57fcfbde354c4 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/AmountAggregate.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/AmountAggregate.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import java.util.Iterator; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/FilteredPartsJoin.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/FilteredPartsJoin.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/FilteredPartsJoin.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/FilteredPartsJoin.java index b21546fa0ca45..8377b21b181cb 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/FilteredPartsJoin.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/FilteredPartsJoin.java @@ -11,11 +11,11 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.JoinFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/IntPair.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/IntPair.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/IntPair.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/IntPair.java index 32ecaa2b4e2b2..901c4e4bc6e53 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/IntPair.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/IntPair.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Pair; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/LineItemMap.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/LineItemMap.java similarity index 94% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/LineItemMap.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/LineItemMap.java index 10876f3ea038e..95f0210d5398f 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/LineItemMap.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/LineItemMap.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/OrderMap.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/OrderMap.java similarity index 93% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/OrderMap.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/OrderMap.java index ac53c53843350..2af3a5b636053 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/OrderMap.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/OrderMap.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/OrderedPartsJoin.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/OrderedPartsJoin.java similarity index 94% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/OrderedPartsJoin.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/OrderedPartsJoin.java index 4b126bdd72610..200b1d4f73d27 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/OrderedPartsJoin.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/OrderedPartsJoin.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.JoinFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartFilter.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartFilter.java similarity index 93% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartFilter.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartFilter.java index 97af696fcfc2e..8a65bd06094ab 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartFilter.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartFilter.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.NullValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartJoin.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartJoin.java similarity index 94% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartJoin.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartJoin.java index 66b09f265c904..a47f3372e506c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartJoin.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartJoin.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.JoinFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartListJoin.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartListJoin.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartListJoin.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartListJoin.java index 92f715b797692..39cd89dab1958 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartListJoin.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartListJoin.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.JoinFunction; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartsuppMap.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartsuppMap.java similarity index 93% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartsuppMap.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartsuppMap.java index 01bfa5242004f..1cac9909367ce 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/PartsuppMap.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/PartsuppMap.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/StringIntPair.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/StringIntPair.java similarity index 95% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/StringIntPair.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/StringIntPair.java index c27ddc6fdaeb9..e365c7b9e2f5c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/StringIntPair.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/StringIntPair.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Pair; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/StringIntPairStringDataOutFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/StringIntPairStringDataOutFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java index b87075c39c4a6..e3421abdb49fa 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/StringIntPairStringDataOutFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/StringIntPairStringDataOutFormat.java @@ -12,7 +12,7 @@ **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import java.io.IOException; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/SupplierMap.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/SupplierMap.java similarity index 93% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/SupplierMap.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/SupplierMap.java index 45070d6cfc449..c31155a82826a 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/SupplierMap.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/SupplierMap.java @@ -11,10 +11,10 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.util.Collector; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/SuppliersJoin.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/SuppliersJoin.java similarity index 93% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/SuppliersJoin.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/SuppliersJoin.java index 8fa9d15a5b2eb..c001e4d6fb6d5 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/tpch9/SuppliersJoin.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query9Util/SuppliersJoin.java @@ -11,11 +11,11 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.tpch9; +package eu.stratosphere.test.recordJobs.relational.query9Util; import eu.stratosphere.api.java.record.functions.JoinFunction; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; import eu.stratosphere.types.StringValue; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/ReduceGroupSort.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/ReduceGroupSort.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/ReduceGroupSort.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/ReduceGroupSort.java index 6608ab7716bde..3ef6f87d6bfeb 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/ReduceGroupSort.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/ReduceGroupSort.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort; +package eu.stratosphere.test.recordJobs.sort; import java.io.Serializable; import java.util.Iterator; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/TeraSort.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/TeraSort.java similarity index 90% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/TeraSort.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/TeraSort.java index 3b7e66e0f1221..52412d637ccfb 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/TeraSort.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/TeraSort.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort; +package eu.stratosphere.test.recordJobs.sort; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; @@ -20,10 +20,10 @@ import eu.stratosphere.api.common.operators.FileDataSource; import eu.stratosphere.api.common.operators.Order; import eu.stratosphere.api.common.operators.Ordering; -import eu.stratosphere.example.java.record.sort.terasort.TeraDistribution; -import eu.stratosphere.example.java.record.sort.terasort.TeraInputFormat; -import eu.stratosphere.example.java.record.sort.terasort.TeraKey; -import eu.stratosphere.example.java.record.sort.terasort.TeraOutputFormat; +import eu.stratosphere.test.recordJobs.sort.tsUtil.TeraDistribution; +import eu.stratosphere.test.recordJobs.sort.tsUtil.TeraInputFormat; +import eu.stratosphere.test.recordJobs.sort.tsUtil.TeraKey; +import eu.stratosphere.test.recordJobs.sort.tsUtil.TeraOutputFormat; /** * This is an example implementation of the well-known TeraSort benchmark using the Stratosphere system. The benchmark diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraDistribution.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraDistribution.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java index f70aeccfe27c8..14f0611d96b47 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraDistribution.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort.terasort; +package eu.stratosphere.test.recordJobs.sort.tsUtil; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraInputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraInputFormat.java similarity index 96% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraInputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraInputFormat.java index ba676f99d2772..1c6ce3f41f8e5 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraInputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraInputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort.terasort; +package eu.stratosphere.test.recordJobs.sort.tsUtil; import eu.stratosphere.api.java.record.io.DelimitedInputFormat; import eu.stratosphere.types.Record; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraKey.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraKey.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java index ccfaedc36ac56..f541a07d64280 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraKey.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort.terasort; +package eu.stratosphere.test.recordJobs.sort.tsUtil; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraOutputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraOutputFormat.java similarity index 96% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraOutputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraOutputFormat.java index 9c58acb8a2301..acd6b76bcc76c 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraOutputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraOutputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort.terasort; +package eu.stratosphere.test.recordJobs.sort.tsUtil; import java.io.IOException; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraValue.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraValue.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java index 268ad66017124..714019c5c578b 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/sort/terasort/TeraValue.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.sort.terasort; +package eu.stratosphere.test.recordJobs.sort.tsUtil; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/util/ConfigUtils.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/ConfigUtils.java similarity index 97% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/util/ConfigUtils.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/ConfigUtils.java index e7d56333be437..3f59f2d2efe46 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/util/ConfigUtils.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/ConfigUtils.java @@ -12,7 +12,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.util; +package eu.stratosphere.test.recordJobs.util; import java.util.HashSet; import java.util.Set; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/DiscardingOutputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/DiscardingOutputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java index e93d84a9c641f..57253d186a025 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/DiscardingOutputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/DiscardingOutputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import java.io.IOException; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/InfiniteIntegerInputFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/InfiniteIntegerInputFormat.java similarity index 96% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/InfiniteIntegerInputFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/InfiniteIntegerInputFormat.java index 556f6ae44914a..3e695c5e90a20 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/InfiniteIntegerInputFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/InfiniteIntegerInputFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import java.io.IOException; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/InfiniteIntegerInputFormatWithDelay.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java similarity index 97% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/InfiniteIntegerInputFormatWithDelay.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java index 2aab361f73bc9..7d9e657418671 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/InfiniteIntegerInputFormatWithDelay.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/InfiniteIntegerInputFormatWithDelay.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import java.io.IOException; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/IntTupleDataInFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/IntTupleDataInFormat.java similarity index 97% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/IntTupleDataInFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/IntTupleDataInFormat.java index a6e676c060bf7..5212b8ebd0e2c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/IntTupleDataInFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/IntTupleDataInFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import eu.stratosphere.api.java.record.io.DelimitedInputFormat; import eu.stratosphere.types.IntValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/StringTupleDataOutFormat.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/StringTupleDataOutFormat.java similarity index 97% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/StringTupleDataOutFormat.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/StringTupleDataOutFormat.java index da8ef4c6b1e04..4735b99b4a33c 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/StringTupleDataOutFormat.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/StringTupleDataOutFormat.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import eu.stratosphere.api.java.record.io.DelimitedOutputFormat; import eu.stratosphere.types.Record; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/Tuple.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java similarity index 99% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/Tuple.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java index 10d613ede315d..d9720e8ebed8b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/Tuple.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/UniformIntInput.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/UniformIntInput.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/UniformIntInput.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/UniformIntInput.java index 160f13a9c100b..3b149aef5b300 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/UniformIntInput.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/UniformIntInput.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.testPrograms.util; +package eu.stratosphere.test.recordJobs.util; import eu.stratosphere.api.java.record.io.GenericInputFormat; import eu.stratosphere.configuration.Configuration; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/AnonymousWordCount.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java similarity index 98% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/AnonymousWordCount.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java index 678a007af60f8..6422fa427c693 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/AnonymousWordCount.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.wordcount; +package eu.stratosphere.test.recordJobs.wordcount; import java.util.Iterator; import java.util.StringTokenizer; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCount.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCount.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCount.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCount.java index 420d53e1ef76d..3dcdf031ab92d 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCount.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCount.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.wordcount; +package eu.stratosphere.test.recordJobs.wordcount; import java.util.Iterator; import java.util.StringTokenizer; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCountAccumulators.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCountAccumulators.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java index 0d24e25995e32..d102246deba8f 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCountAccumulators.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.wordcount; +package eu.stratosphere.test.recordJobs.wordcount; import java.io.DataInput; import java.io.DataOutput; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCountOptimized.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java similarity index 99% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCountOptimized.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java index 1372ad2139c6e..ddd786bc329af 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/record/wordcount/WordCountOptimized.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.record.wordcount; +package eu.stratosphere.test.recordJobs.wordcount; import java.util.Iterator; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/IntTupleDataInFormatTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/IntTupleDataInFormatTest.java index 1b76c39d46f3c..dc0f55f280fb8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/IntTupleDataInFormatTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/IntTupleDataInFormatTest.java @@ -16,8 +16,8 @@ import org.junit.Assert; import org.junit.Test; -import eu.stratosphere.test.testPrograms.util.IntTupleDataInFormat; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.IntTupleDataInFormat; +import eu.stratosphere.test.recordJobs.util.Tuple; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java index eb045043efa04..e43ce90d16d6f 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java @@ -22,7 +22,7 @@ import org.junit.Assert; import org.junit.Test; -import eu.stratosphere.test.testPrograms.util.Tuple; +import eu.stratosphere.test.recordJobs.util.Tuple; public class TupleTest { From e5450b3f4d6c6362611855a8bcf6bc7691cbbc44 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Wed, 14 May 2014 20:00:08 +0200 Subject: [PATCH 017/182] Remove some of the redundant Record examples - some KMeans Variants - some WordCount variants --- .../KMeansIterativeNepheleITCase.java | 8 +- .../iterations/IterativeKMeansTest.java | 6 +- .../plandump/DumpCompiledPlanTest.java | 4 +- .../plandump/PreviewPlanDumpTest.java | 6 +- .../DistributedCacheTest.java | 2 - .../test/iterative/IterativeKMeansITCase.java | 4 +- .../KMeansITCase.java} | 15 +- .../KMeansTutorialExampleITCase.java | 52 --- .../{KMeans.java => KMeansBroadcast.java} | 4 +- ...{KMeansIterative.java => KMeansCross.java} | 4 +- .../recordJobs/kmeans/KMeansSingleStep.java | 221 +++++++++++- .../kmeans/KMeansTutorialExample.java | 329 ------------------ .../wordcount/AnonymousWordCount.java | 101 ------ .../wordcount/WordCountOptimized.java | 160 --------- 14 files changed, 243 insertions(+), 673 deletions(-) rename stratosphere-tests/src/test/java/eu/stratosphere/test/{recordJobTests/KMeansStepITCase.java => iterative/KMeansITCase.java} (81%) delete mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java rename stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/{KMeans.java => KMeansBroadcast.java} (98%) rename stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/{KMeansIterative.java => KMeansCross.java} (97%) delete mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java delete mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java delete mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java index e1d65799e5ae4..aef08e0106bdf 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java @@ -43,10 +43,10 @@ import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.pact.runtime.task.util.TaskConfig; import eu.stratosphere.test.iterative.nephele.JobGraphUtils; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointBuilder; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.RecomputeClusterCenter; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.SelectNearestCenter; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointOutFormat; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.PointBuilder; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.RecomputeClusterCenter; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.SelectNearestCenter; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.PointOutFormat; import eu.stratosphere.test.testdata.KMeansData; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.DoubleValue; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java index d2b2965d8613f..b429f787e7cff 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/iterations/IterativeKMeansTest.java @@ -34,7 +34,7 @@ import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.util.LocalStrategy; import eu.stratosphere.test.compiler.CompilerTestBase; -import eu.stratosphere.test.recordJobs.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; public class IterativeKMeansTest extends CompilerTestBase { @@ -58,7 +58,7 @@ public class IterativeKMeansTest extends CompilerTestBase { @Test public void testCompileKMeansSingleStepWithStats() { - KMeans kmi = new KMeans(); + KMeansBroadcast kmi = new KMeansBroadcast(); Plan p = kmi.getPlan(String.valueOf(DEFAULT_PARALLELISM), IN_FILE, IN_FILE, OUT_FILE, String.valueOf(20)); // set the statistics @@ -77,7 +77,7 @@ public void testCompileKMeansSingleStepWithStats() { @Test public void testCompileKMeansSingleStepWithOutStats() { - KMeans kmi = new KMeans(); + KMeansBroadcast kmi = new KMeansBroadcast(); Plan p = kmi.getPlan(String.valueOf(DEFAULT_PARALLELISM), IN_FILE, IN_FILE, OUT_FILE, String.valueOf(20)); OptimizedPlan plan = compileNoStats(p); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java index 2064b3f6ff67d..46df373d32fa9 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java @@ -26,7 +26,7 @@ import eu.stratosphere.compiler.plandump.PlanJSONDumpGenerator; import eu.stratosphere.example.java.graph.ConnectedComponents; import eu.stratosphere.test.compiler.CompilerTestBase; -import eu.stratosphere.test.recordJobs.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; import eu.stratosphere.test.recordJobs.relational.WebLogAnalysis; @@ -59,7 +59,7 @@ public void dumpWebLogAnalysis() { @Test public void dumpBulkIterationKMeans() { - dump(new KMeans().getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, OUT_FILE)); + dump(new KMeansBroadcast().getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, OUT_FILE)); } @Test diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java index f1cc11ac40791..112b0cdc3e701 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java @@ -28,7 +28,7 @@ import eu.stratosphere.compiler.dag.DataSinkNode; import eu.stratosphere.compiler.plandump.PlanJSONDumpGenerator; import eu.stratosphere.example.java.graph.ConnectedComponents; -import eu.stratosphere.test.recordJobs.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; import eu.stratosphere.test.recordJobs.relational.WebLogAnalysis; @@ -75,8 +75,8 @@ public void dumpWebLogAnalysis() { @Test public void dumpBulkIterationKMeans() { - dump(new KMeans().getPlan("4", IN_FILE, OUT_FILE)); - dump(new KMeans().getPlan(NO_ARGS)); + dump(new KMeansBroadcast().getPlan("4", IN_FILE, OUT_FILE)); + dump(new KMeansBroadcast().getPlan(NO_ARGS)); } @Test diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/distributedCache/DistributedCacheTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/distributedCache/DistributedCacheTest.java index 68904c7a7aeb1..345a82335034b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/distributedCache/DistributedCacheTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/distributedCache/DistributedCacheTest.java @@ -34,8 +34,6 @@ import java.io.FileReader; import java.util.HashSet; import java.util.Set; -import java.util.logging.Level; -import java.util.logging.Logger; /** diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java index d8f46aee7b5bb..c8da45faff88b 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/IterativeKMeansITCase.java @@ -17,7 +17,7 @@ import java.util.List; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.recordJobs.kmeans.KMeans; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; import eu.stratosphere.test.testdata.KMeansData; import eu.stratosphere.test.util.RecordAPITestBase; @@ -37,7 +37,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - KMeans kmi = new KMeans(); + KMeansBroadcast kmi = new KMeansBroadcast(); return kmi.getPlan("4", dataPath, clusterPath, resultPath, "20"); } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/KMeansStepITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java similarity index 81% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/KMeansStepITCase.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java index 195d8a7ccbce2..7085c3c186e04 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/KMeansStepITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansITCase.java @@ -11,22 +11,23 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.test.recordJobTests; +package eu.stratosphere.test.iterative; import java.util.ArrayList; import java.util.List; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; +import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; import eu.stratosphere.test.testdata.KMeansData; import eu.stratosphere.test.util.RecordAPITestBase; -public class KMeansStepITCase extends RecordAPITestBase { + +public class KMeansITCase extends RecordAPITestBase { protected String dataPath; protected String clusterPath; protected String resultPath; - + @Override protected void preSubmit() throws Exception { dataPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS); @@ -36,8 +37,8 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - KMeansSingleStep kmi = new KMeansSingleStep(); - return kmi.getPlan("4", dataPath, clusterPath, resultPath); + KMeansBroadcast kmi = new KMeansBroadcast(); + return kmi.getPlan("4", dataPath, clusterPath, resultPath, "20"); } @@ -46,6 +47,6 @@ protected void postSubmit() throws Exception { List resultLines = new ArrayList(); readAllResultLines(resultLines, resultPath); - KMeansData.checkResultsWithDelta(KMeansData.CENTERS_AFTER_ONE_STEP_SINGLE_DIGIT, resultLines, 0.1); + KMeansData.checkResultsWithDelta(KMeansData.CENTERS_AFTER_20_ITERATIONS_SINGLE_DIGIT, resultLines, 0.1); } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java deleted file mode 100644 index 78e8908a59ab4..0000000000000 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/KMeansTutorialExampleITCase.java +++ /dev/null @@ -1,52 +0,0 @@ -/*********************************************************************************************************************** - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.test.iterative; - -import java.util.ArrayList; -import java.util.List; - -import eu.stratosphere.api.common.Plan; -import eu.stratosphere.test.recordJobs.kmeans.KMeansTutorialExample; -import eu.stratosphere.test.testdata.KMeansData; -import eu.stratosphere.test.util.RecordAPITestBase; - - -public class KMeansTutorialExampleITCase extends RecordAPITestBase { - - protected String dataPath; - protected String clusterPath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS_2D); - clusterPath = createTempFile("initial_centers.txt", KMeansData.INITIAL_CENTERS_2D); - resultPath = getTempDirPath("result"); - } - - @Override - protected Plan getTestJob() { - KMeansTutorialExample kmi = new KMeansTutorialExample(); - return kmi.getPlan("4", dataPath, clusterPath, resultPath, "20"); - } - - - @Override - protected void postSubmit() throws Exception { - List resultLines = new ArrayList(); - readAllResultLines(resultLines, resultPath + "/" + KMeansTutorialExample.CENTERS_RESULT_SUBDIRECTORY); - - KMeansData.checkResultsWithDelta(KMeansData.CENTERS_2D_AFTER_20_ITERATIONS_DOUBLE_DIGIT, resultLines, 0.02); - } -} diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeans.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java similarity index 98% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeans.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java index cca940aa50dac..bde54c939f6da 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeans.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java @@ -44,7 +44,7 @@ import eu.stratosphere.util.Collector; -public class KMeans implements Program, ProgramDescription { +public class KMeansBroadcast implements Program, ProgramDescription { private static final long serialVersionUID = 1L; @@ -314,6 +314,6 @@ public void writeRecord(Record record) throws IOException { } public static void main(String[] args) throws Exception { - System.out.println(LocalExecutor.optimizerPlanAsJSON(new KMeans().getPlan("4", "/dev/random", "/dev/random", "/tmp", "20"))); + System.out.println(LocalExecutor.optimizerPlanAsJSON(new KMeansBroadcast().getPlan("4", "/dev/random", "/dev/random", "/tmp", "20"))); } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansIterative.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansCross.java similarity index 97% rename from stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansIterative.java rename to stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansCross.java index 4451e77feae19..4d450cb950eb7 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansIterative.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansCross.java @@ -34,7 +34,7 @@ import eu.stratosphere.types.IntValue; -public class KMeansIterative implements Program, ProgramDescription { +public class KMeansCross implements Program, ProgramDescription { private static final long serialVersionUID = 1L; @@ -116,7 +116,7 @@ public String getDescription() { } public static void main(String[] args) throws Exception { - KMeansIterative kmi = new KMeansIterative(); + KMeansCross kmi = new KMeansCross(); if (args.length < 5) { System.err.println(kmi.getDescription()); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java index 500b6be221b44..28ad7cb5d6522 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java @@ -14,20 +14,32 @@ package eu.stratosphere.test.recordJobs.kmeans; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; import eu.stratosphere.api.common.ProgramDescription; import eu.stratosphere.api.common.operators.FileDataSink; import eu.stratosphere.api.common.operators.FileDataSource; +import eu.stratosphere.api.java.record.functions.MapFunction; +import eu.stratosphere.api.java.record.functions.ReduceFunction; import eu.stratosphere.api.java.record.io.CsvInputFormat; +import eu.stratosphere.api.java.record.io.FileOutputFormat; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointBuilder; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.PointOutFormat; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.RecomputeClusterCenter; -import eu.stratosphere.test.recordJobs.kmeans.KMeans.SelectNearestCenter; +import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; +import eu.stratosphere.configuration.Configuration; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; +import eu.stratosphere.types.Record; +import eu.stratosphere.types.Value; +import eu.stratosphere.util.Collector; public class KMeansSingleStep implements Program, ProgramDescription { @@ -80,4 +92,205 @@ public Plan getPlan(String... args) { public String getDescription() { return "Parameters: "; } + + public static final class Point implements Value { + private static final long serialVersionUID = 1L; + + public double x, y, z; + + public Point() {} + + public Point(double x, double y, double z) { + this.x = x; + this.y = y; + this.z = z; + } + + public void add(Point other) { + x += other.x; + y += other.y; + z += other.z; + } + + public Point div(long val) { + x /= val; + y /= val; + z /= val; + return this; + } + + public double euclideanDistance(Point other) { + return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y) + (z-other.z)*(z-other.z)); + } + + public void clear() { + x = y = z = 0.0; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeDouble(x); + out.writeDouble(y); + out.writeDouble(z); + } + + @Override + public void read(DataInput in) throws IOException { + x = in.readDouble(); + y = in.readDouble(); + z = in.readDouble(); + } + + @Override + public String toString() { + return "(" + x + "|" + y + "|" + z + ")"; + } + } + + public static final class PointWithId { + + public int id; + public Point point; + + public PointWithId(int id, Point p) { + this.id = id; + this.point = p; + } + } + + /** + * Determines the closest cluster center for a data point. + */ + public static final class SelectNearestCenter extends MapFunction { + private static final long serialVersionUID = 1L; + + private final IntValue one = new IntValue(1); + private final Record result = new Record(3); + + private List centers = new ArrayList(); + + /** + * Reads all the center values from the broadcast variable into a collection. + */ + @Override + public void open(Configuration parameters) throws Exception { + Collection clusterCenters = this.getRuntimeContext().getBroadcastVariable("centers"); + + centers.clear(); + for (Record r : clusterCenters) { + centers.add(new PointWithId(r.getField(0, IntValue.class).getValue(), r.getField(1, Point.class))); + } + } + + /** + * Computes a minimum aggregation on the distance of a data point to cluster centers. + * + * Output Format: + * 0: centerID + * 1: pointVector + * 2: constant(1) (to enable combinable average computation in the following reducer) + */ + @Override + public void map(Record dataPointRecord, Collector out) { + Point p = dataPointRecord.getField(1, Point.class); + + double nearestDistance = Double.MAX_VALUE; + int centerId = -1; + + // check all cluster centers + for (PointWithId center : centers) { + // compute distance + double distance = p.euclideanDistance(center.point); + + // update nearest cluster if necessary + if (distance < nearestDistance) { + nearestDistance = distance; + centerId = center.id; + } + } + + // emit a new record with the center id and the data point. add a one to ease the + // implementation of the average function with a combiner + result.setField(0, new IntValue(centerId)); + result.setField(1, p); + result.setField(2, one); + + out.collect(result); + } + } + + @Combinable + public static final class RecomputeClusterCenter extends ReduceFunction { + private static final long serialVersionUID = 1L; + + private final Point p = new Point(); + + + /** + * Compute the new position (coordinate vector) of a cluster center. + */ + @Override + public void reduce(Iterator points, Collector out) { + Record sum = sumPointsAndCount(points); + sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); + out.collect(sum); + } + + /** + * Computes a pre-aggregated average value of a coordinate vector. + */ + @Override + public void combine(Iterator points, Collector out) { + out.collect(sumPointsAndCount(points)); + } + + private final Record sumPointsAndCount(Iterator dataPoints) { + Record next = null; + p.clear(); + int count = 0; + + // compute coordinate vector sum and count + while (dataPoints.hasNext()) { + next = dataPoints.next(); + p.add(next.getField(1, Point.class)); + count += next.getField(2, IntValue.class).getValue(); + } + + next.setField(1, p); + next.setField(2, new IntValue(count)); + return next; + } + } + + public static final class PointBuilder extends MapFunction { + + private static final long serialVersionUID = 1L; + + @Override + public void map(Record record, Collector out) throws Exception { + double x = record.getField(1, DoubleValue.class).getValue(); + double y = record.getField(2, DoubleValue.class).getValue(); + double z = record.getField(3, DoubleValue.class).getValue(); + + record.setField(1, new Point(x, y, z)); + out.collect(record); + } + } + + public static final class PointOutFormat extends FileOutputFormat { + + private static final long serialVersionUID = 1L; + + private static final String format = "%d|%.1f|%.1f|%.1f|\n"; + + @Override + public void writeRecord(Record record) throws IOException { + int id = record.getField(0, IntValue.class).getValue(); + Point p = record.getField(1, Point.class); + + byte[] bytes = String.format(format, id, p.x, p.y, p.z).getBytes(); + + this.stream.write(bytes); + } + } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java deleted file mode 100644 index dd3155b2f35d1..0000000000000 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansTutorialExample.java +++ /dev/null @@ -1,329 +0,0 @@ -/*********************************************************************************************************************** - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.test.recordJobs.kmeans; - - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; - -import eu.stratosphere.api.common.Plan; -import eu.stratosphere.api.common.Program; -import eu.stratosphere.api.common.ProgramDescription; -import eu.stratosphere.api.common.operators.BulkIteration; -import eu.stratosphere.api.common.operators.FileDataSink; -import eu.stratosphere.api.common.operators.FileDataSource; -import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.api.java.record.functions.ReduceFunction; -import eu.stratosphere.api.java.record.io.CsvInputFormat; -import eu.stratosphere.api.java.record.io.FileOutputFormat; -import eu.stratosphere.api.java.record.operators.MapOperator; -import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; -import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.types.DoubleValue; -import eu.stratosphere.types.IntValue; -import eu.stratosphere.types.Record; -import eu.stratosphere.types.Value; -import eu.stratosphere.util.Collector; - - -public class KMeansTutorialExample implements Program, ProgramDescription { - - public static final String CENTERS_RESULT_SUBDIRECTORY = "centers"; - public static final String POINTS_RESULT_SUBDIRECTORY = "points"; - - private static final long serialVersionUID = 1L; - - @Override - @SuppressWarnings("unchecked") - public Plan getPlan(String... args) { - - // parse job parameters - final int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - final String dataPointInput = (args.length > 1 ? args[1] : ""); - final String clusterInput = (args.length > 2 ? args[2] : ""); - final String output = (args.length > 3 ? args[3] : ""); - final int numIterations = (args.length > 4 ? Integer.parseInt(args[4]) : 2); - - final String centersOutput = output + "/" + CENTERS_RESULT_SUBDIRECTORY; - final String taggedPointsOutput = output + "/" + POINTS_RESULT_SUBDIRECTORY; - - // create DataSourceContract for data point input - FileDataSource pointsSource = new FileDataSource(new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class), dataPointInput, "Data Points"); - - // create DataSourceContract for cluster center input - FileDataSource clustersSource = new FileDataSource(new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class), clusterInput, "Centers"); - - MapOperator dataPoints = MapOperator.builder(new PointBuilder()).name("Build data points").input(pointsSource).build(); - - MapOperator clusterPoints = MapOperator.builder(new PointBuilder()).name("Build cluster points").input(clustersSource).build(); - - // ---------------------- Begin K-Means Loop --------------------- - - BulkIteration iter = new BulkIteration("k-means loop"); - iter.setInput(clusterPoints); - iter.setMaximumNumberOfIterations(numIterations); - - // compute the distances and select the closest center - MapOperator findNearestClusterCenters = MapOperator.builder(new SelectNearestCenter()) - .setBroadcastVariable("centers", iter.getPartialSolution()) - .input(dataPoints) - .name("Find Nearest Centers") - .build(); - - // computing the new cluster positions - ReduceOperator recomputeClusterCenter = ReduceOperator.builder(new RecomputeClusterCenter(), IntValue.class, 0) - .input(findNearestClusterCenters) - .name("Recompute Center Positions") - .build(); - - iter.setNextPartialSolution(recomputeClusterCenter); - - // ---------------------- End K-Means Loop --------------------- - - - // run one additional points-to-centers assignment, to have the points tagged with the final centers they belong to - // create DataSourceContract for data point input - - MapOperator findNearestFinalCluster = MapOperator.builder(new SelectNearestCenter()) - .setBroadcastVariable("centers", iter) - .input(dataPoints) - .name("Tag points with final centroid") - .build(); - - // write the new cluster positions - FileDataSink newClusterPoints = new FileDataSink(new PointOutFormat(), centersOutput, iter, "Cluster Positions"); - - // write assigned clusters - FileDataSink clusterAssignments = new FileDataSink(new PointOutFormat(), taggedPointsOutput, findNearestFinalCluster, "Cluster Assignments"); - - // return the plan - Plan plan = new Plan(newClusterPoints, "KMeans Iteration"); - plan.addDataSink(clusterAssignments); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - @Override - public String getDescription() { - return "Parameters: "; - } - - // -------------------------------------------------------------------------------------------- - // Data Types and UDFs - // -------------------------------------------------------------------------------------------- - - /** - * A simple two-dimensional point. - */ - public static final class Point implements Value { - private static final long serialVersionUID = 1L; - - public double x, y; - - public Point() {} - - public Point(double x, double y) { - this.x = x; - this.y = y; - } - - public void add(Point other) { - x += other.x; - y += other.y; - } - - public Point div(long val) { - x /= val; - y /= val; - return this; - } - - public double euclideanDistance(Point other) { - return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y)); - } - - public void clear() { - x = y = 0.0; - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeDouble(x); - out.writeDouble(y); - } - - @Override - public void read(DataInput in) throws IOException { - x = in.readDouble(); - y = in.readDouble(); - } - - @Override - public String toString() { - return "(" + x + "|" + y + ")"; - } - } - - public static final class PointWithId { - - public int id; - public Point point; - - public PointWithId(int id, Point p) { - this.id = id; - this.point = p; - } - } - - /** - * Determines the closest cluster center for a data point. - */ - public static final class SelectNearestCenter extends MapFunction { - private static final long serialVersionUID = 1L; - - private final IntValue one = new IntValue(1); - private final Record result = new Record(3); - - private List centers = new ArrayList(); - - /** - * Reads all the center values from the broadcast variable into a collection. - */ - @Override - public void open(Configuration parameters) throws Exception { - Collection clusterCenters = this.getRuntimeContext().getBroadcastVariable("centers"); - - centers.clear(); - for (Record r : clusterCenters) { - centers.add(new PointWithId(r.getField(0, IntValue.class).getValue(), r.getField(1, Point.class))); - } - } - - /** - * Computes a minimum aggregation on the distance of a data point to cluster centers. - * - * Output Format: - * 0: centerID - * 1: pointVector - * 2: constant(1) (to enable combinable average computation in the following reducer) - */ - @Override - public void map(Record dataPointRecord, Collector out) { - Point p = dataPointRecord.getField(1, Point.class); - - double nearestDistance = Double.MAX_VALUE; - int centerId = -1; - - // check all cluster centers - for (PointWithId center : centers) { - // compute distance - double distance = p.euclideanDistance(center.point); - - // update nearest cluster if necessary - if (distance < nearestDistance) { - nearestDistance = distance; - centerId = center.id; - } - } - - // emit a new record with the center id and the data point. add a one to ease the - // implementation of the average function with a combiner - result.setField(0, new IntValue(centerId)); - result.setField(1, p); - result.setField(2, one); - - out.collect(result); - } - } - - @Combinable - public static final class RecomputeClusterCenter extends ReduceFunction { - private static final long serialVersionUID = 1L; - - private final Point p = new Point(); - - - /** - * Compute the new position (coordinate vector) of a cluster center. - */ - @Override - public void reduce(Iterator points, Collector out) { - Record sum = sumPointsAndCount(points); - sum.setField(1, sum.getField(1, Point.class).div(sum.getField(2, IntValue.class).getValue())); - out.collect(sum); - } - - /** - * Computes a pre-aggregated average value of a coordinate vector. - */ - @Override - public void combine(Iterator points, Collector out) { - out.collect(sumPointsAndCount(points)); - } - - private final Record sumPointsAndCount(Iterator dataPoints) { - Record next = null; - p.clear(); - int count = 0; - - // compute coordinate vector sum and count - while (dataPoints.hasNext()) { - next = dataPoints.next(); - p.add(next.getField(1, Point.class)); - count += next.getField(2, IntValue.class).getValue(); - } - - next.setField(1, p); - next.setField(2, new IntValue(count)); - return next; - } - } - - public static final class PointBuilder extends MapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void map(Record record, Collector out) throws Exception { - double x = record.getField(1, DoubleValue.class).getValue(); - double y = record.getField(2, DoubleValue.class).getValue(); - - record.setField(1, new Point(x, y)); - out.collect(record); - } - } - - public static final class PointOutFormat extends FileOutputFormat { - - private static final long serialVersionUID = 1L; - - private static final String format = "%d|%.2f|%.2f|\n"; - - @Override - public void writeRecord(Record record) throws IOException { - int id = record.getField(0, IntValue.class).getValue(); - Point p = record.getField(1, Point.class); - - byte[] bytes = String.format(format, id, p.x, p.y).getBytes(); - - this.stream.write(bytes); - } - } -} diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java deleted file mode 100644 index 6422fa427c693..0000000000000 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/AnonymousWordCount.java +++ /dev/null @@ -1,101 +0,0 @@ -/*********************************************************************************************************************** - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.test.recordJobs.wordcount; - -import java.util.Iterator; -import java.util.StringTokenizer; - -import eu.stratosphere.api.common.Plan; -import eu.stratosphere.api.common.Program; -import eu.stratosphere.api.common.operators.FileDataSink; -import eu.stratosphere.api.common.operators.FileDataSource; -import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.api.java.record.functions.ReduceFunction; -import eu.stratosphere.api.java.record.io.CsvOutputFormat; -import eu.stratosphere.api.java.record.io.TextInputFormat; -import eu.stratosphere.api.java.record.operators.MapOperator; -import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.types.IntValue; -import eu.stratosphere.types.Record; -import eu.stratosphere.types.StringValue; -import eu.stratosphere.util.Collector; - -/** - * Implements a word count which takes the input file and counts the number of - * the occurrences of each word in the file. - */ -public class AnonymousWordCount implements Program { - - private static final long serialVersionUID = 1L; - - @Override - public Plan getPlan(String... args) { - // parse job parameters - int defaultParallelism = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String inputPath = (args.length > 1 ? args[1] : ""); - String outputPath = (args.length > 2 ? args[2] : ""); - - FileDataSource source = new FileDataSource(new TextInputFormat(), inputPath); - - MapOperator mapper = MapOperator.builder(new MapFunction() { - - private static final long serialVersionUID = 1L; - - public void map(Record record, Collector collector) throws Exception { - String line = record.getField(0, StringValue.class).getValue(); - - // normalize the line - line = line.replaceAll("\\W+", " ").toLowerCase(); - - // tokenize the line - StringTokenizer tokenizer = new StringTokenizer(line); - while (tokenizer.hasMoreTokens()) { - String word = tokenizer.nextToken(); - - // we emit a (word, 1) pair - collector.collect(new Record(new StringValue(word), new IntValue(1))); - } - } - }).input(source).build(); - - ReduceOperator reducer = ReduceOperator.builder(new ReduceFunction() { - - private static final long serialVersionUID = 1L; - - public void reduce(Iterator records, Collector collector) { - Record element = null; - int sum = 0; - - while (records.hasNext()) { - element = records.next(); - int cnt = element.getField(1, IntValue.class).getValue(); - sum += cnt; - } - - element.setField(1, new IntValue(sum)); - collector.collect(element); - } - }).keyField(StringValue.class, 0).input(mapper).build(); - - FileDataSink out = new FileDataSink(new CsvOutputFormat(), outputPath, reducer, "Word Counts"); - CsvOutputFormat.configureRecordFormat(out) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(out, "WordCount Example", defaultParallelism); - return plan; - } -} diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java deleted file mode 100644 index ddd786bc329af..0000000000000 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountOptimized.java +++ /dev/null @@ -1,160 +0,0 @@ -/*********************************************************************************************************************** - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.test.recordJobs.wordcount; - -import java.util.Iterator; - -import eu.stratosphere.api.common.Plan; -import eu.stratosphere.api.common.Program; -import eu.stratosphere.api.common.ProgramDescription; -import eu.stratosphere.api.common.operators.FileDataSink; -import eu.stratosphere.api.common.operators.FileDataSource; -import eu.stratosphere.api.java.record.functions.FunctionAnnotation.ConstantFields; -import eu.stratosphere.api.java.record.functions.MapFunction; -import eu.stratosphere.api.java.record.functions.ReduceFunction; -import eu.stratosphere.api.java.record.io.CsvOutputFormat; -import eu.stratosphere.api.java.record.io.TextInputFormat; -import eu.stratosphere.api.java.record.operators.MapOperator; -import eu.stratosphere.api.java.record.operators.ReduceOperator; -import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; -import eu.stratosphere.client.LocalExecutor; -import eu.stratosphere.types.IntValue; -import eu.stratosphere.types.Record; -import eu.stratosphere.types.StringValue; -import eu.stratosphere.util.Collector; -import eu.stratosphere.util.SimpleStringUtils; - -/** - * Implements a word count which takes the input file and counts the number of - * the occurrences of each word in the file. Compared to the {@link WordCount} example. - * this program performs better by using mutable objects and optimized tools. - */ -public class WordCountOptimized implements Program, ProgramDescription { - - private static final long serialVersionUID = 1L; - - /** - * Converts a Record containing one string in to multiple string/integer pairs. - * The string is tokenized by whitespaces. For each token a new record is emitted, - * where the token is the first field and an Integer(1) is the second field. - */ - public static class TokenizeLine extends MapFunction { - - private static final long serialVersionUID = 1L; - - // initialize reusable mutable objects - private final Record outputRecord = new Record(); - private StringValue word = new StringValue(); - private final IntValue one = new IntValue(1); - - private final SimpleStringUtils.WhitespaceTokenizer tokenizer = - new SimpleStringUtils.WhitespaceTokenizer(); - - @Override - public void map(Record record, Collector collector) { - // get the first field (as type StringValue) from the record - StringValue line = record.getField(0, StringValue.class); - - // normalize the line - SimpleStringUtils.replaceNonWordChars(line, ' '); - SimpleStringUtils.toLowerCase(line); - - // tokenize the line - this.tokenizer.setStringToTokenize(line); - while (tokenizer.next(this.word)) { - // we emit a (word, 1) pair - this.outputRecord.setField(0, this.word); - this.outputRecord.setField(1, this.one); - collector.collect(this.outputRecord); - } - } - } - - /** - * Sums up the counts for a certain given key. The counts are assumed to be at position 1 - * in the record. The other fields are not modified. - */ - @Combinable - @ConstantFields(0) - public static class CountWords extends ReduceFunction { - - private static final long serialVersionUID = 1L; - - private final IntValue cnt = new IntValue(); - - @Override - public void reduce(Iterator records, Collector out) throws Exception { - Record element = null; - int sum = 0; - while (records.hasNext()) { - element = records.next(); - IntValue i = element.getField(1, IntValue.class); - sum += i.getValue(); - } - - this.cnt.setValue(sum); - element.setField(1, this.cnt); - out.collect(element); - } - - @Override - public void combine(Iterator records, Collector out) throws Exception { - // the logic is the same as in the reduce function, so simply call the reduce method - this.reduce(records, out); - } - } - - - @Override - public Plan getPlan(String... args) { - // parse job parameters - int numSubTasks = (args.length > 0 ? Integer.parseInt(args[0]) : 1); - String dataInput = (args.length > 1 ? args[1] : ""); - String output = (args.length > 2 ? args[2] : ""); - - FileDataSource source = new FileDataSource(TextInputFormat.class, dataInput, "Input Lines"); - source.setParameter(TextInputFormat.CHARSET_NAME, "ASCII"); // comment out this line for UTF-8 inputs - MapOperator mapper = MapOperator.builder(TokenizeLine.class) - .input(source) - .name("Tokenize Lines") - .build(); - ReduceOperator reducer = ReduceOperator.builder(CountWords.class, StringValue.class, 0) - .input(mapper) - .name("Count Words") - .build(); - FileDataSink out = new FileDataSink(CsvOutputFormat.class, output, reducer, "Word Counts"); - CsvOutputFormat.configureRecordFormat(out) - .recordDelimiter('\n') - .fieldDelimiter(' ') - .field(StringValue.class, 0) - .field(IntValue.class, 1); - - Plan plan = new Plan(out, "WordCount Example"); - plan.setDefaultParallelism(numSubTasks); - return plan; - } - - - @Override - public String getDescription() { - return "Parameters: [numSubStasks] [input] [output]"; - } - - // This can be used to locally run a plan from within eclipse (or anywhere else) - public static void main(String[] args) throws Exception { - WordCountOptimized wc = new WordCountOptimized(); - Plan plan = wc.getPlan("1", "file:///path/to/input", "file:///path/to/output"); - LocalExecutor.execute(plan); - } -} From 2a14a05daae01dd10d80634e0c3b403963031915 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Wed, 14 May 2014 21:51:01 +0200 Subject: [PATCH 018/182] Add javaDoc comments to ExecutionEnvironment (final), DataSet (parts), InputTypeConfigurable, and ResultTypeQueryable. --- .../eu/stratosphere/api/java/DataSet.java | 17 +- .../api/java/ExecutionEnvironment.java | 256 +++++++++++++++++- .../java/typeutils/InputTypeConfigurable.java | 12 +- .../java/typeutils/ResultTypeQueryable.java | 11 +- 4 files changed, 277 insertions(+), 19 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java index 288c2a6ca6621..7709e6693be52 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java @@ -582,6 +582,13 @@ public DeltaIteration iterateDelta(DataSet workset, int maxIteratio // ------------------------------------------------------------------------------------------- + /** + * Runs a {@link CustomUnaryOperation} on the data set. Custom operations are typically complex + * operators that are composed of multiple steps. + * + * @param operation The operation to run. + * @return The data set produced by the operation. + */ public DataSet runOperation(CustomUnaryOperation operation) { Validate.notNull(operation, "The custom operator must not be null."); operation.setInput(this); @@ -684,6 +691,7 @@ public DataSink printToErr() { /** * Writes a DataSet using a {@link FileOutputFormat} to a specified location. + * This method adds a data sink to the program. * * @param outputFormat The FileOutputFormat to write the DataSet. * @param filePath The path to the location where the DataSet is written. @@ -701,6 +709,7 @@ public DataSink write(FileOutputFormat outputFormat, String filePath) { /** * Writes a DataSet using a {@link FileOutputFormat} to a specified location. + * This method adds a data sink to the program. * * @param outputFormat The FileOutputFormat to write the DataSet. * @param filePath The path to the location where the DataSet is written. @@ -720,10 +729,12 @@ public DataSink write(FileOutputFormat outputFormat, String filePath, Writ } /** - * Writes a DataSet using an {@link OutputFormat}. + * Processes a DataSet using an {@link OutputFormat}. This method adds a data sink to the program. + * Programs may have multiple data sinks. A DataSet may also have multiple consumers (data sinks + * or transformations) at the same time. * - * @param outputFormat The OutputFormat to write the DataSet. - * @return The DataSink that writes the DataSet. + * @param outputFormat The OutputFormat to process the DataSet. + * @return The DataSink that processes the DataSet. * * @see OutputFormat * @see DataSink diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java index 1513fe8737e37..e832f3ae4ff8e 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java @@ -52,14 +52,23 @@ import eu.stratosphere.util.NumberSequenceIterator; import eu.stratosphere.util.SplittableIterator; - /** * The ExecutionEnviroment is the context in which a program is executed. A * {@link LocalEnvironment} will cause execution in the current JVM, a * {@link RemoteEnvironment} will cause execution on a remote setup. *

- * The environment provides methods to control the job execution (such as - * setting the parallelism) and to interact with the outside world (data access). + * The environment provides methods to control the job execution (such as setting the parallelism) + * and to interact with the outside world (data access). + *

+ * Please note that the execution environment needs strong type information for the input and return types + * of all operations that are executed. This means that the environments needs to know that the return + * value of an operation is for example a Tuple of String and Integer. + * Because the Java compiler throws much of the generic type information away, most methods attempt to re- + * obtain that information using reflection. In certain cases, it may be necessary to manually supply that + * information to some of the methods. + * + * @see LocalEnvironment + * @see RemoteEnvironment */ public abstract class ExecutionEnvironment { @@ -229,16 +238,36 @@ public DataSource readTextFileWithValue(String filePath, String cha // ----------------------------------- CSV Input Format --------------------------------------- - public CsvReader readCsvFile(Path filePath) { - return new CsvReader(filePath, this); - } - + /** + * Creates a CSV reader to read a comma separated value (CSV) file. The reader has options to + * define parameters and field types and will eventually produce the DataSet that corresponds to + * the read and parsed CSV input. + * + * @param filePath The path of the CSV file. + * @return A CsvReader that can be used to configure the CSV input. + */ public CsvReader readCsvFile(String filePath) { return new CsvReader(filePath, this); } // ----------------------------------- Generic Input Format --------------------------------------- + /** + * Generic method to create an input DataSet with in {@link InputFormat}. The DataSet will not be + * immediately created - instead, this method returns a DataSet that will be lazily created from + * the input format once the program is executed. + *

+ * Since all data sets need specific information about their types, this method needs to determine + * the type of the data produced by the input format. It will attempt to determine the data type + * by reflection, unless the the input format implements the {@link ResultTypeQueryable} interface. + * In the latter case, this method will invoke the {@link ResultTypeQueryable#getProducedType()} + * method to determine data type produced by the input format. + * + * @param inputFormat The input format used to create the data set. + * @return A DataSet that represents the data created by the input format. + * + * @see #createInput(InputFormat, TypeInformation) + */ public DataSource createInput(InputFormat inputFormat) { if (inputFormat == null) { throw new IllegalArgumentException("InputFormat must not be null."); @@ -257,7 +286,21 @@ public DataSource createInput(InputFormat inputFormat) { "Please specify the TypeInformation of the produced type explicitly."); } } - + + /** + * Generic method to create an input DataSet with in {@link InputFormat}. The DataSet will not be + * immediately created - instead, this method returns a DataSet that will be lazily created from + * the input format once the program is executed. + *

+ * The data set is typed to the given TypeInformation. This method is intended for input formats that + * where the return type cannot be determined by reflection analysis, and that do not implement the + * {@link ResultTypeQueryable} interface. + * + * @param inputFormat The input format used to create the data set. + * @return A DataSet that represents the data created by the input format. + * + * @see #createInput(InputFormat) + */ public DataSource createInput(InputFormat inputFormat, TypeInformation producedType) { if (inputFormat == null) { throw new IllegalArgumentException("InputFormat must not be null."); @@ -272,6 +315,24 @@ public DataSource createInput(InputFormat inputFormat, TypeInformat // ----------------------------------- Collection --------------------------------------- + /** + * Creates a DataSet from the given non-empty collection. The type of the data set is that + * of the elements in the collection. The elements need to be serializable (as defined by + * {@link java.io.Serializable}), because the framework may move the elements into the cluster + * if needed. + *

+ * The framework will try and determine the exact type from the collection elements. + * In case of generic elements, it may be necessary to manually supply the type information + * via {@link #fromCollection(Collection, TypeInformation)}. + *

+ * Note that this operation will result in a non-parallel data source, i.e. a data source with + * a degree of parallelism of one. + * + * @param data The collection of elements to create the data set from. + * @return A DataSet representing the given collection. + * + * @see #fromCollection(Collection, TypeInformation) + */ public DataSource fromCollection(Collection data) { if (data == null) { throw new IllegalArgumentException("The data must not be null."); @@ -285,17 +346,70 @@ public DataSource fromCollection(Collection data) { return fromCollection(data, TypeExtractor.getForObject(firstValue)); } - + /** + * Creates a DataSet from the given non-empty collection. The type of the data set is that + * of the elements in the collection. The elements need to be serializable (as defined by + * {@link java.io.Serializable}), because the framework may move the elements into the cluster + * if needed. + *

+ * Note that this operation will result in a non-parallel data source, i.e. a data source with + * a degree of parallelism of one. + *

+ * The returned DataSet is typed to the given TypeInformation. + * + * @param data The collection of elements to create the data set from. + * @param type The TypeInformation for the produced data set. + * @return A DataSet representing the given collection. + * + * @see #fromCollection(Collection) + */ public DataSource fromCollection(Collection data, TypeInformation type) { CollectionInputFormat.checkCollection(data, type.getTypeClass()); return new DataSource(this, new CollectionInputFormat(data), type); } + /** + * Creates a DataSet from the given iterator. Because the iterator will remain unmodified until + * the actual execution happens, the type of data returned by the iterator must be given + * explicitly in the form of the type class (this is due to the fact that the Java compiler + * erases the generic type information). + *

+ * The iterator must be serializable (as defined in {@link java.io.Serializable}), because the + * framework may move it to a remote environment, if needed. + *

+ * Note that this operation will result in a non-parallel data source, i.e. a data source with + * a degree of parallelism of one. + * + * @param data The collection of elements to create the data set from. + * @param type The class of the data produced by the iterator. Must not be a generic class. + * @return A DataSet representing the elements in the iterator. + * + * @see #fromCollection(Iterator, TypeInformation) + */ public DataSource fromCollection(Iterator data, Class type) { return fromCollection(data, TypeExtractor.getForClass(type)); } + /** + * Creates a DataSet from the given iterator. Because the iterator will remain unmodified until + * the actual execution happens, the type of data returned by the iterator must be given + * explicitly in the form of the type information. This method is useful for cases where the type + * is generic. In that case, the type class (as given in {@link #fromCollection(Iterator, Class)} + * does not supply all type information. + *

+ * The iterator must be serializable (as defined in {@link java.io.Serializable}), because the + * framework may move it to a remote environment, if needed. + *

+ * Note that this operation will result in a non-parallel data source, i.e. a data source with + * a degree of parallelism of one. + * + * @param data The collection of elements to create the data set from. + * @param type The TypeInformation for the produced data set. + * @return A DataSet representing the elements in the iterator. + * + * @see #fromCollection(Iterator, Class) + */ public DataSource fromCollection(Iterator data, TypeInformation type) { if (!(data instanceof Serializable)) { throw new IllegalArgumentException("The iterator must be serializable."); @@ -310,9 +424,16 @@ public DataSource fromCollection(Iterator data, TypeInformation typ * for example, all of the {@link String} or {@link Integer}. The sequence of elements must not be empty. * Furthermore, the elements must be serializable (as defined in {@link java.io.Serializable}, because the * execution environment may ship the elements into the cluster. + *

+ * The framework will try and determine the exact type from the collection elements. + * In case of generic elements, it may be necessary to manually supply the type information + * via {@link #fromCollection(Collection, TypeInformation)}. + *

+ * Note that this operation will result in a non-parallel data source, i.e. a data source with + * a degree of parallelism of one. * * @param data The elements to make up the data set. - * @return A data set representing the given list of elements. + * @return A DataSet representing the given list of elements. */ public DataSource fromElements(X... data) { if (data == null) { @@ -326,16 +447,55 @@ public DataSource fromElements(X... data) { } + /** + * Creates a new data set that contains elements in the iterator. The iterator is splittable, allowing the + * framework to create a parallel data source that returns the elements in the iterator. + * The iterator must be serializable (as defined in {@link java.io.Serializable}, because the + * execution environment may ship the elements into the cluster. + *

+ * Because the iterator will remain unmodified until the actual execution happens, the type of data + * returned by the iterator must be given explicitly in the form of the type class (this is due to the + * fact that the Java compiler erases the generic type information). + * + * @param iterator The iterator that produces the elements of the data set. + * @param type The class of the data produced by the iterator. Must not be a generic class. + * @return A DataSet representing the elements in the iterator. + * + * @see #fromParallelCollection(SplittableIterator, TypeInformation) + */ public DataSource fromParallelCollection(SplittableIterator iterator, Class type) { return fromParallelCollection(iterator, TypeExtractor.getForClass(type)); } - + /** + * Creates a new data set that contains elements in the iterator. The iterator is splittable, allowing the + * framework to create a parallel data source that returns the elements in the iterator. + * The iterator must be serializable (as defined in {@link java.io.Serializable}, because the + * execution environment may ship the elements into the cluster. + *

+ * Because the iterator will remain unmodified until the actual execution happens, the type of data + * returned by the iterator must be given explicitly in the form of the type information. + * This method is useful for cases where the type is generic. In that case, the type class + * (as given in {@link #fromParallelCollection(SplittableIterator, Class)} does not supply all type information. + * + * @param iterator The iterator that produces the elements of the data set. + * @param type The TypeInformation for the produced data set. + * @return A DataSet representing the elements in the iterator. + * + * @see #fromParallelCollection(SplittableIterator, Class) + */ public DataSource fromParallelCollection(SplittableIterator iterator, TypeInformation type) { return new DataSource(this, new ParallelIteratorInputFormat(iterator), type); } - + /** + * Creates a new data set that contains a sequence of numbers. The data set will be created in parallel, + * so there is no guarantee about the oder of the elements. + * + * @param from The number to start at (inclusive). + * @param to The number to stop at (inclusive). + * @return A DataSet, containing all number in the {@code [from, to]} interval. + */ public DataSource generateSequence(long from, long to) { return fromParallelCollection(new NumberSequenceIterator(from, to), BasicTypeInfo.LONG_TYPE_INFO); } @@ -344,28 +504,100 @@ public DataSource generateSequence(long from, long to) { // Executing // -------------------------------------------------------------------------------------------- + /** + * Triggers the program execution. The environment will execute all parts of the program that have + * resulted in a "sink" operation. Sink operations are for example printing results ({@link DataSet#print()}, + * writing results (e.g. {@link DataSet#writeAsText(String)}, + * {@link DataSet#write(eu.stratosphere.api.common.io.FileOutputFormat, String)}, or other generic + * data sinks created with {@link DataSet#output(eu.stratosphere.api.common.io.OutputFormat)}. + *

+ * The program execution will be logged and displayed with a generated default name. + * + * @return The result of the job execution, containing elapsed time and accumulators. + * @throws Exception Thrown, if the program executions fails. + */ public JobExecutionResult execute() throws Exception { return execute(getDefaultName()); } + /** + * Triggers the program execution. The environment will execute all parts of the program that have + * resulted in a "sink" operation. Sink operations are for example printing results ({@link DataSet#print()}, + * writing results (e.g. {@link DataSet#writeAsText(String)}, + * {@link DataSet#write(eu.stratosphere.api.common.io.FileOutputFormat, String)}, or other generic + * data sinks created with {@link DataSet#output(eu.stratosphere.api.common.io.OutputFormat)}. + *

+ * The program execution will be logged and displayed with the given job name. + * + * @return The result of the job execution, containing elapsed time and accumulators. + * @throws Exception Thrown, if the program executions fails. + */ public abstract JobExecutionResult execute(String jobName) throws Exception; + /** + * Creates the plan with which the system will execute the program, and returns it as + * a String using a JSON representation of the execution data flow graph. + * + * @return The execution plan of the program, as a JSON String. + * @throws Exception Thrown, if the compiler could not be instantiated, or the master could not + * be contacted to retrieve information relevant to the execution planning. + */ public abstract String getExecutionPlan() throws Exception; + /** + * Registers a file at the distributed cache under the given name. The file will be accessible + * from any user-defined function in the (distributed) runtime under a local path. Files + * may be local files, or files in a distributed file system. The runtime will copy the files + * temporarily to a local cache, if needed. + *

+ * The {@link eu.stratosphere.api.common.functions.RuntimeContext} can be obtained inside UDFs via + * {@link eu.stratosphere.api.common.functions.Function#getRuntimeContext()} and provides access + * {@link eu.stratosphere.api.common.cache.DistributedCache} via + * {@link eu.stratosphere.api.common.functions.RuntimeContext#getDistributedCache()}. + * + * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") + * @param name The name under which the file is registered. + */ public void registerCachedFile(String filePath, String name){ this.cacheFile.add(new Tuple2(filePath, name)); } + /** + * Registers all files that were registered at this execution environment's cache registry of the + * given plan's cache registry. + * + * @param p The plan to register files at. + * @throws IOException Thrown if checks for existence and sanity fail. + */ protected void registerCachedFilesWithPlan(Plan p) throws IOException { for (Tuple2 entry : cacheFile) { p.registerCachedFile(entry.f0, entry.f1); } } + /** + * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, + * and operations and how they interact, as an isolated unit that can be executed with a + * {@link eu.stratosphere.api.common.PlanExecutor}. Obtaining a plan and starting it with an + * executor is an alternative way to run a program and is only possible if the program consists + * only of distributed operations. + * + * @return The program's plan. + */ public JavaPlan createProgramPlan() { return createProgramPlan(null); } + /** + * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, + * and operations and how they interact, as an isolated unit that can be executed with a + * {@link eu.stratosphere.api.common.PlanExecutor}. Obtaining a plan and starting it with an + * executor is an alternative way to run a program and is only possible if the program consists + * only of distributed operations. + * + * @param jobName The name attached to the plan (displayed in logs and monitoring). + * @return The program's plan. + */ public JavaPlan createProgramPlan(String jobName) { if (this.sinks.isEmpty()) { throw new RuntimeException("No data sinks have been created yet. A program needs at least one sink that consumes data. Examples are writing the data set or printing it."); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/InputTypeConfigurable.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/InputTypeConfigurable.java index 96d4187e30c96..7852d9fef3792 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/InputTypeConfigurable.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/InputTypeConfigurable.java @@ -14,11 +14,19 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.typeutils; - /** - * + * {@link eu.stratosphere.api.common.io.OutputFormat}s can implement this interface to be configured + * with the data type they will operate on. The method {@link #setInputType(TypeInformation)} will be + * called when the output format is used with an output method such as + * {@link eu.stratosphere.api.java.DataSet#output(eu.stratosphere.api.common.io.OutputFormat)}. */ public interface InputTypeConfigurable { + /** + * Method that is called on an {@link eu.stratosphere.api.common.io.OutputFormat} when it is passed to + * the DataSet's output method. May be used to configures the output format based on the data type. + * + * @param type The data type of the input. + */ void setInputType(TypeInformation type); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ResultTypeQueryable.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ResultTypeQueryable.java index 2babdc6b8582a..7713ce35bb3de 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ResultTypeQueryable.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ResultTypeQueryable.java @@ -15,10 +15,17 @@ package eu.stratosphere.api.java.typeutils; /** - * Interface to be implemented by functions and input formats who be queries for the - * {@link TypeInformation} of the result. + * This interface can be implemented by functions and input formats to tell the framework + * about their produced data type. This method acts as an alternative to the reflection analysis + * that is otherwise performed and is useful in situations where the produced data type may vary + * depending on parameterization. */ public interface ResultTypeQueryable { + /** + * Gets the data type (as a {@link TypeInformation}) produced by this function or input format. + * + * @return The data type produced by this function or input format. + */ TypeInformation getProducedType(); } From 3d0e3a701c4cccb7eef0e4fe4a2ad2e995fc04a0 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 12 May 2014 14:25:12 +0200 Subject: [PATCH 019/182] Expose WriteMode in JAPI. #781 Pass Configuration parameters to runtime operators in JAPI (+test) --- pom.xml | 1 + .../api/common/io/FileOutputFormat.java | 1 - .../eu/stratosphere/api/java/DataSet.java | 45 +++++++++++++++++-- .../java/operators/OperatorTranslation.java | 16 +++++++ .../test/javaApiOperators/MapITCase.java | 41 ++++++++++++++++- 5 files changed, 98 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 9279069f7c1bc..a52433cee115f 100644 --- a/pom.xml +++ b/pom.xml @@ -320,6 +320,7 @@ org.apache.maven.plugins maven-jar-plugin + 2.4 diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java index 6e14660dffd14..50d3d17dfe704 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/FileOutputFormat.java @@ -120,7 +120,6 @@ public FileOutputFormat(Path outputPath) { this.outputFilePath = outputPath; } - public void setOutputFilePath(Path path) { if (path == null) { throw new IllegalArgumentException("Output file path may not be null."); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java index 7709e6693be52..ec4bfd3337cd6 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java @@ -630,6 +630,22 @@ public DataSink writeAsText(String filePath) { return output(new TextOutputFormat(new Path(filePath))); } + /** + * Writes a DataSet as a text file to the specified location.
+ * For each element of the DataSet the result of {@link Object#toString()} is written. + * + * @param filePath The path pointing to the location the text file is written to. + * @param writeMode Control the behavior for existing files. Options are NO_OVERWRITE and OVERWRITE. + * @return The DataSink that writes the DataSet. + * + * @see TextOutputFormat + */ + public DataSink writeAsText(String filePath, WriteMode writeMode) { + TextOutputFormat tof = new TextOutputFormat(new Path(filePath)); + tof.setWriteMode(writeMode); + return output(tof); + } + /** * Writes a {@link Tuple} DataSet as a CSV file to the specified location.
* Note: Only a Tuple DataSet can written as a CSV file.
@@ -660,13 +676,34 @@ public DataSink writeAsCsv(String filePath) { * @see CsvOutputFormat */ public DataSink writeAsCsv(String filePath, String rowDelimiter, String fieldDelimiter) { - Validate.isTrue(this.type.isTupleType(), "The writeAsCsv() method can only be used on data sets of tuples."); - return internalWriteAsCsv(new Path(filePath), rowDelimiter, fieldDelimiter); + return internalWriteAsCsv(new Path(filePath), rowDelimiter, fieldDelimiter, null); } + /** + * Writes a {@link Tuple} DataSet as a CSV file to the specified location with the specified field and line delimiters.
+ * Note: Only a Tuple DataSet can written as a CSV file.
+ * For each Tuple field the result of {@link Object#toString()} is written. + * + * @param filePath The path pointing to the location the CSV file is written to. + * @param rowDelimiter The row delimiter to separate Tuples. + * @param fieldDelimiter The field delimiter to separate Tuple fields. + * @param writeMode Control the behavior for existing files. Options are NO_OVERWRITE and OVERWRITE. + * + * @see Tuple + * @see CsvOutputFormat + */ + public DataSink writeAsCsv(String filePath, String rowDelimiter, String fieldDelimiter, WriteMode writeMode) { + return internalWriteAsCsv(new Path(filePath), rowDelimiter, fieldDelimiter, writeMode); + } + @SuppressWarnings("unchecked") - private DataSink internalWriteAsCsv(Path filePath, String rowDelimiter, String fieldDelimiter) { - return output((OutputFormat) new CsvOutputFormat(filePath, rowDelimiter, fieldDelimiter)); + private DataSink internalWriteAsCsv(Path filePath, String rowDelimiter, String fieldDelimiter, WriteMode wm) { + Validate.isTrue(this.type.isTupleType(), "The writeAsCsv() method can only be used on data sets of tuples."); + CsvOutputFormat of = new CsvOutputFormat(filePath, rowDelimiter, fieldDelimiter); + if(wm != null) { + of.setWriteMode(wm); + } + return output((OutputFormat) of); } /** diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java index 8b05f88208f89..2555294c77fb3 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java @@ -26,6 +26,7 @@ import eu.stratosphere.api.java.operators.translation.JavaPlan; import eu.stratosphere.api.java.operators.translation.PlanBulkIterationOperator; import eu.stratosphere.api.java.operators.translation.PlanDeltaIterationOperator; +import eu.stratosphere.configuration.Configuration; import java.util.ArrayList; import java.util.HashMap; @@ -84,6 +85,13 @@ else if (dataSet instanceof SingleInputOperator) { Operator input = translate(op.getInput()); // translate the operation itself and connect it to the input dataFlowOp = op.translateToDataFlow(input); + + if(dataSet instanceof UdfOperator ) { + Configuration opParams = ((UdfOperator) op).getParameters(); + if(opParams != null) { + dataFlowOp.getParameters().addAll(opParams); + } + } } else if (dataSet instanceof TwoInputOperator) { TwoInputOperator op = (TwoInputOperator) dataSet; @@ -94,6 +102,14 @@ else if (dataSet instanceof TwoInputOperator) { // translate the operation itself and connect it to the inputs dataFlowOp = op.translateToDataFlow(input1, input2); + + // set configuration params + if(dataSet instanceof UdfOperator ) { + Configuration opParams = ((UdfOperator) op).getParameters(); + if(opParams != null) { + dataFlowOp.getParameters().addAll(opParams); + } + } } else if (dataSet instanceof BulkIterationResultSet) { dataFlowOp = translateBulkIteration((BulkIterationResultSet) dataSet); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java index 8598c4b46b4ea..01531f25e7131 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java @@ -19,6 +19,8 @@ import java.util.Collection; import java.util.LinkedList; +import junit.framework.Assert; + import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -35,7 +37,7 @@ @RunWith(Parameterized.class) public class MapITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 8; + private static int NUM_PROGRAMS = 9; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -439,6 +441,43 @@ public Tuple3 map(Tuple3 value) "55,6,Comment#14\n" + "55,6,Comment#15\n"; } + case 9: { + /* + * Test passing configuration object. + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); + Configuration conf = new Configuration(); + final String testKey = "testVariable"; + final int testValue = 666; + conf.setInteger(testKey, testValue); + DataSet> bcMapDs = ds. + map(new MapFunction, Tuple3>() { + private static final long serialVersionUID = 1L; + private final Tuple3 out = new Tuple3(); + + @Override + public void open(Configuration config) { + int val = config.getInteger(testKey, -1); + Assert.assertEquals(testValue, val); + } + + @Override + public Tuple3 map(Tuple3 value) + throws Exception { + return value; + } + }).withParameters(conf); + bcMapDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1,Hi\n" + + "2,2,Hello\n" + + "3,2,Hello world"; + } default: throw new IllegalArgumentException("Invalid program id"); } From cc197cf95af4b2fe72a049501a25dfc3264d4541 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Tue, 13 May 2014 10:06:37 +0200 Subject: [PATCH 020/182] Refactored and unifid Java API example jobs --- .../BroadcastVariableExample.java | 72 --- .../example/java/clustering/KMeans.java | 242 ++++++++-- .../java/clustering/util/KMeansData.java | 79 ++++ .../KMeansDataGenerator.java} | 83 ++-- .../java/graph/ConnectedComponents.java | 117 ++++- .../EnumTrianglesBasic.java | 177 ++++--- .../EnumTrianglesOpt.java | 208 +++++---- .../example/java/graph/PageRankBasic.java | 247 ++++++++++ .../example/java/graph/SimplePageRank.java | 154 ------- .../graph/util/ConnectedComponentsData.java | 53 +++ .../util/EnumTrianglesData.java} | 7 +- .../util/EnumTrianglesDataTypes.java} | 19 +- .../example/java/graph/util/PageRankData.java | 81 ++++ .../pagerank/SimpleDeltaPageRank.java | 157 ------- .../example/java/relational/TPCHQuery10.java | 265 ++++++----- .../example/java/relational/TPCHQuery3.java | 432 +++++++++--------- .../java/relational/WebLogAnalysis.java | 268 +++++++---- .../java/relational/util/WebLogData.java | 401 ++++++++++++++++ .../WebLogDataGenerator.java} | 121 ++--- .../example/java/wordcount/WordCount.java | 104 +++-- .../java/wordcount/WordCountCollection.java | 75 --- .../java/wordcount/util/WordCountData.java | 62 +++ .../test/testdata/EnumTriangleData.java | 22 +- .../test/testdata/PageRankData.java | 48 +- .../plandump/DumpCompiledPlanTest.java | 17 +- .../plandump/PreviewPlanDumpTest.java | 20 +- .../EnumTriangleBasicITCase.java | 2 +- .../EnumTriangleOptITCase.java | 2 +- .../exampleJavaPrograms/PageRankITCase.java | 8 +- .../exampleJavaPrograms/WordCountITCase.java | 2 +- .../javaApiOperators/DeltaPageRankITCase.java | 150 ------ 31 files changed, 2218 insertions(+), 1477 deletions(-) delete mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/broadcastvar/BroadcastVariableExample.java create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/{generator/KMeansSampleDataGenerator.java => util/KMeansDataGenerator.java} (66%) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/{triangles => graph}/EnumTrianglesBasic.java (55%) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/{triangles => graph}/EnumTrianglesOpt.java (68%) create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java delete mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/SimplePageRank.java create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/ConnectedComponentsData.java rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/{triangles/util/EdgeData.java => graph/util/EnumTrianglesData.java} (89%) rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/{triangles/util/EdgeDataTypes.java => graph/util/EnumTrianglesDataTypes.java} (88%) create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/PageRankData.java delete mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/incremental/pagerank/SimpleDeltaPageRank.java create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogData.java rename stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/{generator/WebLogGenerator.java => util/WebLogDataGenerator.java} (67%) delete mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountCollection.java create mode 100644 stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/util/WordCountData.java delete mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DeltaPageRankITCase.java diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/broadcastvar/BroadcastVariableExample.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/broadcastvar/BroadcastVariableExample.java deleted file mode 100644 index c39b72fcdfbab..0000000000000 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/broadcastvar/BroadcastVariableExample.java +++ /dev/null @@ -1,72 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.example.java.broadcastvar; - -import java.util.Collection; -import java.util.HashSet; -import java.util.Set; - -import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.api.java.functions.MapFunction; -import eu.stratosphere.configuration.Configuration; - -@SuppressWarnings("serial") -public class BroadcastVariableExample { - - public static class ToUppercaseMapper extends MapFunction { - // Lookup table for Strings to uppercase - private Set toUppercase; - - @Override - public void open(Configuration parameters) throws Exception { - // You can access broadcast variables via `getRuntimeContext().getBroadcastVariable(String)`. - // - // The broadcasted variable is registered under the previously provided name and the data set is accessed - // as a Collection over the broadcasted data set (where T is the type of the broadcasted DataSet). - Collection broadcastedData = getRuntimeContext().getBroadcastVariable("toUppercase"); - - this.toUppercase = new HashSet(broadcastedData); - } - - @Override - public String map(String value) throws Exception { - return this.toUppercase.contains(value) ? value.toUpperCase() : value; - } - } - - public static void main(String[] args) throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // This example program takes the data set `lorem` and uppercases every String, which is also included in the - // `toUppercase` data set. - // - // The `toUppercase` data set is BROADCASTED to the map operator, which creates a lookup table from it. The - // lookup tables is then used in the map method to decide whether to uppercase a given String or not. - - DataSet toUppercase = env.fromElements("lorem", "ipsum"); - - DataSet lorem = env.fromElements("lorem", "ipsum", "dolor", "sit", "amet"); - - lorem.map(new ToUppercaseMapper()) - // You can broadcast a data set to an operator via `withBroadcastSet(DataSet, String)`. - // - // The broadcast variable will be registered at the operator under the provided name. - .withBroadcastSet(toUppercase, "toUppercase") - .print(); - - env.execute("Broadcast Variable Example"); - } -} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java index 690c02b5622ce..f0e0f43a142d2 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java @@ -22,15 +22,102 @@ import eu.stratosphere.api.java.IterativeDataSet; import eu.stratosphere.api.java.functions.MapFunction; import eu.stratosphere.api.java.functions.ReduceFunction; +import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; import eu.stratosphere.configuration.Configuration; +import eu.stratosphere.example.java.clustering.util.KMeansData; - +/** + * This example implements a basic K-Means clustering algorithm. + * + *

+ * K-Means is an iterative clustering algorithm and works as follows:
+ * K-Means is given a set of data points to be clustered and an initial set of K cluster centers. + * In each iteration, the algorithm computes the distance of each data point to each cluster center. + * Each point is assigned to the cluster center which is closest to it. + * Subsequently, each cluster center is moved to the center (mean) of all points that have been assigned to it. + * The moved cluster centers are fed into the next iteration. + * The algorithm terminates after a fixed number of iteration (as in this implementation) + * or if cluster centers do not (significantly) move in an iteration. + * + *

+ * This implementation works on two-dimensional data points.
+ * It computes an assignment of data points to cluster centers, i.e., + * each data point is annotated with the id of the final cluster (center) it belongs to. + * + *

+ * Input files are plain text files must be formatted as follows: + *

    + *
  • Data points are represented as two double values separated by a blank character. + * Data points are separated by newline characters.
    + * For example "1.2 2.3\n5.3 7.2\n" gives two data points (x=1.2, y=2.3) and (x=5.3, y=7.2). + *
  • Cluster centers are represented by an integer id and a point value.
    + * For example "1 6.2 3.2\n2 2.9 5.7\n" gives two centers (id=1, x=6.2, y=3.2) and (id=2, x=2.9, y=5.7). + *
+ * + *

+ * This example shows how to use: + *

    + *
  • Bulk iterations + *
  • Broadcast variables in bulk iterations + *
  • Custom Java objects (PoJos) + *
+ */ @SuppressWarnings("serial") public class KMeans { + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + parseParameters(args); + + // set up execution environment + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // get input data + DataSet points = getPointDataSet(env); + DataSet centroids = getCentroidDataSet(env); + + // set number of bulk iterations for KMeans algorithm + IterativeDataSet loop = centroids.iterate(numIterations); + + DataSet newCentriods = points + // compute closest centroid for each point + .map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids") + // count and sum point coordinates for each centroid + .map(new CountAppender()) + .groupBy(0).reduce(new CentroidAccumulator()) + // compute new centroids from point counts and coordinate sums + .map(new CentroidAverager()); + + // feed new centroids back into next iteration + DataSet finalCentroids = loop.closeWith(newCentriods); + + DataSet> clusteredPoints = points + // assign points to final clusters + .map(new SelectNearestCenter()).withBroadcastSet(finalCentroids, "centroids"); + + // emit result + if(fileOutput) { + clusteredPoints.writeAsCsv(outputPath, "\n", ","); + } else { + clusteredPoints.print(); + } + + // execute program + env.execute("KMeans Example"); + + } + + // ************************************************************************* + // DATA TYPES + // ************************************************************************* + /** - * A simple three-dimensional point. + * A simple two-dimensional point. */ public static class Point implements Serializable { @@ -65,10 +152,13 @@ public void clear() { @Override public String toString() { - return "(" + x + "|" + y + ")"; + return x + "," + y; } } + /** + * A simple two-dimensional centroid, basically a point with an ID. + */ public static class Centroid extends Point { public int id; @@ -84,27 +174,50 @@ public Centroid(int id, Point p) { super(p.x, p.y); this.id = id; } + + @Override + public String toString() { + return id + "," + super.toString(); + } } - /** - * Determines the closest cluster center for a data point. - */ - public static final class SelectNearestCenter extends MapFunction> { - private static final long serialVersionUID = 1L; + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** Converts a Tuple2 into a Point. */ + public static final class TuplePointConverter extends MapFunction, Point> { + @Override + public Point map(Tuple2 t) throws Exception { + return new Point(t.f0, t.f1); + } + } + + /** Converts a Tuple3 into a Centroid. */ + public static final class TupleCentroidConverter extends MapFunction, Centroid> { + + @Override + public Centroid map(Tuple3 t) throws Exception { + return new Centroid(t.f0, t.f1, t.f2); + } + } + + /** Determines the closest cluster center for a data point. */ + public static final class SelectNearestCenter extends MapFunction> { private Collection centroids; - /** Reads the centroid values from the broadcast variable into a collection.*/ + /** Reads the centroid values from a broadcast variable into a collection. */ @Override public void open(Configuration parameters) throws Exception { this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); } - + @Override - public Tuple3 map(Point p) throws Exception { + public Tuple2 map(Point p) throws Exception { - double nearestDistance = Double.MAX_VALUE; - int centroidId = 0; + double minDistance = Double.MAX_VALUE; + int closestCentroidId = -1; // check all cluster centers for (Centroid centroid : centroids) { @@ -112,32 +225,37 @@ public Tuple3 map(Point p) throws Exception { double distance = p.euclideanDistance(centroid); // update nearest cluster if necessary - if (distance < nearestDistance) { - nearestDistance = distance; - centroidId = centroid.id; + if (distance < minDistance) { + minDistance = distance; + closestCentroidId = centroid.id; } } // emit a new record with the center id and the data point. - return new Tuple3(centroidId, p, 1L); + return new Tuple2(closestCentroidId, p); } } + /** Appends a count variable to the tuple. */ + public static final class CountAppender extends MapFunction, Tuple3> { + + @Override + public Tuple3 map(Tuple2 t) { + return new Tuple3(t.f0, t.f1, 1L); + } + } - /** The input and output types are (centroid-id, point-sum, count) */ - public static class CentroidAccumulator extends ReduceFunction> { - private static final long serialVersionUID = 1L; + /** Sums and counts point coordinates. */ + public static final class CentroidAccumulator extends ReduceFunction> { @Override public Tuple3 reduce(Tuple3 val1, Tuple3 val2) { return new Tuple3(val1.f0, val1.f1.add(val2.f1), val1.f2 + val2.f2); } } - - /** The input and output types are (centroid-id, point-sum, count) */ - public static class CentroidAverager extends MapFunction, Centroid> { - private static final long serialVersionUID = 1L; + /** Computes new centroid from coordinate sum and count of points. */ + public static final class CentroidAverager extends MapFunction, Centroid> { @Override public Centroid map(Tuple3 value) { @@ -145,30 +263,60 @@ public Centroid map(Tuple3 value) { } } + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* - public static void main(String[] args) throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet points = env.fromElements(new Point(-3.78, -42.01), new Point(-45.96, 30.67), new Point(8.96, -41.58), - new Point(-22.96, 40.73), new Point(4.79, -35.58), new Point(-41.27, 32.42), - new Point(-2.61, -30.43), new Point(-23.33, 26.23), new Point(-9.22, -31.23), - new Point(-45.37, 36.42)); - - DataSet centroids = env.fromElements(new Centroid(0, 43.28, 47.89), - new Centroid(1, -0.06, -48.97)); - - IterativeDataSet loop = centroids.iterate(20); - - DataSet newCentriods = points - .map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids") - .groupBy(0).reduce(new CentroidAccumulator()) - .map(new CentroidAverager()); - - DataSet result = loop.closeWith(newCentriods); - - result.print(); + private static boolean fileOutput = false; + private static String pointsPath = null; + private static String centersPath = null; + private static String outputPath = null; + private static int numIterations = 10; + + private static void parseParameters(String[] programArguments) { - env.execute("KMeans 2d example"); + if(programArguments.length > 0) { + // parse input arguments + fileOutput = true; + if(programArguments.length == 4) { + pointsPath = programArguments[0]; + centersPath = programArguments[1]; + outputPath = programArguments[2]; + numIterations = Integer.parseInt(programArguments[3]); + } else { + System.err.println("Usage: KMeans "); + System.exit(1); + } + } else { + System.out.println("Executing K-Means example with default parameters and built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" Usage: KMeans "); + } + } + + private static DataSet getPointDataSet(ExecutionEnvironment env) { + if(fileOutput) { + // read points from CSV file + return env.readCsvFile(pointsPath) + .fieldDelimiter(' ') + .includeFields(true, true) + .types(Double.class, Double.class) + .map(new TuplePointConverter()); + } else { + return KMeansData.getDefaultPointDataSet(env); + } } + + private static DataSet getCentroidDataSet(ExecutionEnvironment env) { + if(fileOutput) { + return env.readCsvFile(centersPath) + .fieldDelimiter(' ') + .includeFields(true, true, true) + .types(Integer.class, Double.class, Double.class) + .map(new TupleCentroidConverter()); + } else { + return KMeansData.getDefaultCentroidDataSet(env); + } + } + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java new file mode 100644 index 0000000000000..5b37002c93db4 --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java @@ -0,0 +1,79 @@ +/*********************************************************************************************************************** + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.clustering.util; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.example.java.clustering.KMeans.Centroid; +import eu.stratosphere.example.java.clustering.KMeans.Point; + +public class KMeansData { + + public static DataSet getDefaultCentroidDataSet(ExecutionEnvironment env) { + + return env.fromElements( + new Centroid(1, -31.85, -44.77), + new Centroid(2, 35.16, 17.46), + new Centroid(3, -5.16, 21.93), + new Centroid(4, -24.06, 6.81) + ); + } + + public static DataSet getDefaultPointDataSet(ExecutionEnvironment env) { + + return env.fromElements( + new Point(-14.22, -48.01), + new Point(-22.78, 37.10), + new Point(56.18, -42.99), + new Point(35.04, 50.29), + new Point(-9.53, -46.26), + new Point(-34.35, 48.25), + new Point(55.82, -57.49), + new Point(21.03, 54.64), + new Point(-13.63, -42.26), + new Point(-36.57, 32.63), + new Point(50.65, -52.40), + new Point(24.48, 34.04), + new Point(-2.69, -36.02), + new Point(-38.80, 36.58), + new Point(24.00, -53.74), + new Point(32.41, 24.96), + new Point(-4.32, -56.92), + new Point(-22.68, 29.42), + new Point(59.02, -39.56), + new Point(24.47, 45.07), + new Point(5.23, -41.20), + new Point(-23.00, 38.15), + new Point(44.55, -51.50), + new Point(14.62, 59.06), + new Point(7.41, -56.05), + new Point(-26.63, 28.97), + new Point(47.37, -44.72), + new Point(29.07, 51.06), + new Point(0.59, -31.89), + new Point(-39.09, 20.78), + new Point(42.97, -48.98), + new Point(34.36, 49.08), + new Point(-21.91, -49.01), + new Point(-46.68, 46.04), + new Point(48.52, -43.67), + new Point(30.05, 49.25), + new Point(4.03, -43.56), + new Point(-37.85, 41.72), + new Point(38.24, -48.32), + new Point(20.83, 57.85) + ); + } + +} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/generator/KMeansSampleDataGenerator.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansDataGenerator.java similarity index 66% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/generator/KMeansSampleDataGenerator.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansDataGenerator.java index 37c2f1e95fc17..9754edcb4dace 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/generator/KMeansSampleDataGenerator.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansDataGenerator.java @@ -11,7 +11,7 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.clustering.generator; +package eu.stratosphere.example.java.clustering.util; import java.io.BufferedWriter; import java.io.File; @@ -21,35 +21,48 @@ import java.util.Locale; import java.util.Random; -public class KMeansSampleDataGenerator { +import eu.stratosphere.example.java.clustering.KMeans; + +/** + * Generates data for the {@link KMeans} example program. + */ +public class KMeansDataGenerator { static { Locale.setDefault(Locale.US); } private static final String CENTERS_FILE = "centers"; - private static final String POINTS_FILE = "points"; - private static final long DEFAULT_SEED = 4650285087650871364L; - private static final double DEFAULT_VALUE_RANGE = 100.0; - private static final double RELATIVE_STDDEV = 0.08; - private static final int DIMENSIONALITY = 2; - - private static final DecimalFormat FORMAT = new DecimalFormat("#0.00"); - - private static final char DELIMITER = '|'; - - + private static final char DELIMITER = ' '; + /** + * Main method to generate data for the {@link KMeans} example program. + *

+ * The generator creates to files: + *

    + *
  • {tmp.dir}/points for the data points + *
  • {tmp.dir}/centers for the cluster centers + *
+ * + * @param args + *
    + *
  1. Int: Number of data points + *
  2. Int: Number of cluster centers + *
  3. Optional Double: Standard deviation of data points + *
  4. Optional Double: Value range of cluster centers + *
  5. Optional Long: Random seed + *
+ */ public static void main(String[] args) throws IOException { // check parameter count - if (args.length < 3) { + if (args.length < 2) { System.out.println("KMeansDataGenerator [] [] []"); System.exit(1); } @@ -57,23 +70,21 @@ public static void main(String[] args) throws IOException { // parse parameters final int numDataPoints = Integer.parseInt(args[0]); final int k = Integer.parseInt(args[1]); - final double stddev = args.length > 2 ? Double.parseDouble(args[2]) : RELATIVE_STDDEV; final double range = args.length > 3 ? Double.parseDouble(args[4]) : DEFAULT_VALUE_RANGE; final long firstSeed = args.length > 4 ? Long.parseLong(args[4]) : DEFAULT_SEED; - // generate the centers first final double absoluteStdDev = stddev * range; final Random random = new Random(firstSeed); + final String tmpDir = System.getProperty("java.io.tmpdir"); - - // the means for our gaussian distributions + // the means around which data points are distributed final double[][] means = uniformRandomCenters(random, k, DIMENSIONALITY, range); // write the points out BufferedWriter pointsOut = null; try { - pointsOut = new BufferedWriter(new FileWriter(new File(POINTS_FILE))); + pointsOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+POINTS_FILE))); StringBuilder buffer = new StringBuilder(); double[] point = new double[DIMENSIONALITY]; @@ -85,7 +96,7 @@ public static void main(String[] args) throws IOException { for (int d = 0; d < DIMENSIONALITY; d++) { point[d] = (random.nextGaussian() * absoluteStdDev) + centroid[d]; } - write(i, point, buffer, pointsOut); + writePoint(point, buffer, pointsOut); nextCentroid = (nextCentroid + 1) % k; } } @@ -94,18 +105,17 @@ public static void main(String[] args) throws IOException { pointsOut.close(); } } - // write the uniformly distributed centers to a file BufferedWriter centersOut = null; try { - centersOut = new BufferedWriter(new FileWriter(new File(CENTERS_FILE))); + centersOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+CENTERS_FILE))); StringBuilder buffer = new StringBuilder(); double[][] centers = uniformRandomCenters(random, k, DIMENSIONALITY, range); for (int i = 0; i < k; i++) { - write(i + 1, centers[i], buffer, centersOut); + writeCenter(i + 1, centers[i], buffer, centersOut); } } finally { @@ -113,6 +123,9 @@ public static void main(String[] args) throws IOException { centersOut.close(); } } + + System.out.println("Wrote "+numDataPoints+" data points to "+tmpDir+"/"+POINTS_FILE); + System.out.println("Wrote "+k+" cluster centers to "+tmpDir+"/"+CENTERS_FILE); } private static final double[][] uniformRandomCenters(Random rnd, int num, int dimensionality, double range) { @@ -127,16 +140,34 @@ private static final double[][] uniformRandomCenters(Random rnd, int num, int di return points; } - private static void write(long id, double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException { + private static void writePoint(double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException { buffer.setLength(0); + // write coordinates + for (int j = 0; j < coordinates.length; j++) { + buffer.append(FORMAT.format(coordinates[j])); + if(j < coordinates.length - 1) { + buffer.append(DELIMITER); + } + } + + out.write(buffer.toString()); + out.newLine(); + } + + private static void writeCenter(long id, double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException { + buffer.setLength(0); + + // write id buffer.append(id); buffer.append(DELIMITER); - // append all coordinates + // write coordinates for (int j = 0; j < coordinates.length; j++) { buffer.append(FORMAT.format(coordinates[j])); - buffer.append('|'); + if(j < coordinates.length - 1) { + buffer.append(DELIMITER); + } } out.write(buffer.toString()); diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java index 5fec88f0ae1b2..29c55f210bfec 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java @@ -23,6 +23,7 @@ import eu.stratosphere.api.java.functions.MapFunction; import eu.stratosphere.api.java.tuple.Tuple1; import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.example.java.graph.util.ConnectedComponentsData; import eu.stratosphere.util.Collector; /** @@ -35,33 +36,41 @@ * their current component ids, and the workset as the changed vertices. Because we see all vertices initially as * changed, the initial workset and the initial solution set are identical. Also, the delta to the solution set * is consequently also the next workset. + * + *

+ * Input files are plain text files must be formatted as follows: + *

    + *
  • Vertexes represented as IDs and separated by new-line characters.
    + * For example "1\n2\n12\n42\n63\n" gives five vertices (1), (2), (12), (42), and (63). + *
  • Edges are represented as pairs for vertex IDs which are separated by space + * characters. Edges are separated by new-line characters.
    + * For example "1 2\n2 12\n1 12\n42 63\n" gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63). + *
+ * + *

+ * This example shows how to use: + *

    + *
  • Delta Iterations + *
  • Generic-typed Functions + *
*/ @SuppressWarnings("serial") public class ConnectedComponents implements ProgramDescription { + // ************************************************************************* + // PROGRAM + // ************************************************************************* + public static void main(String... args) throws Exception { - if (args.length < 4) { - System.err.println("Parameters: "); - return; - } - final int maxIterations = Integer.parseInt(args[3]); + parseParameters(args); + // set up execution environment ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet vertices = env.readCsvFile(args[0]).types(Long.class).map(new MapFunction, Long>() { - public Long map(Tuple1 value) { return value.f0; } }); - - DataSet> edges = env.readCsvFile(args[1]).fieldDelimiter(' ').types(Long.class, Long.class); - - DataSet> result = doConnectedComponents(vertices, edges, maxIterations); - - result.writeAsCsv(args[2], "\n", " "); - env.execute("Connected Components"); - } - - - public static DataSet> doConnectedComponents(DataSet vertices, DataSet> edges, int maxIterations) { + // read vertex and edge data + DataSet vertices = getVertexDataSet(env); + DataSet> edges = getEdgeDataSet(env); // assign the initial components (equal to the vertex id. DataSet> verticesWithInitialId = vertices.map(new DuplicateValue()); @@ -77,9 +86,24 @@ public static DataSet> doConnectedComponents(DataSet ve .flatMap(new ComponentIdFilter()); // close the delta iteration (delta and new workset are identical) - return iteration.closeWith(changes, changes); + DataSet> result = iteration.closeWith(changes, changes); + + // emit result + if(fileOutput) { + result.writeAsCsv(outputPath, "\n", " "); + } else { + result.print(); + } + + // execute program + env.execute("Connected Components Example"); + } + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + /** * Function that turns a value into a 2-tuple where both fields are that value. */ @@ -121,4 +145,59 @@ public void flatMap(Tuple2, Tuple2> value, Collec public String getDescription() { return "Parameters: "; } + + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static boolean fileOutput = false; + private static String verticesPath = null; + private static String edgesPath = null; + private static String outputPath = null; + private static int maxIterations = 10; + + private static void parseParameters(String[] programArguments) { + + if(programArguments.length > 0) { + // parse input arguments + fileOutput = true; + if(programArguments.length == 4) { + verticesPath = programArguments[0]; + edgesPath = programArguments[1]; + outputPath = programArguments[2]; + maxIterations = Integer.parseInt(programArguments[3]); + } else { + System.err.println("Usage: ConnectedComponents "); + System.exit(1); + } + } else { + System.out.println("Executing Connected Components example with default parameters and built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" Usage: ConnectedComponents "); + } + } + + private static DataSet getVertexDataSet(ExecutionEnvironment env) { + + if(fileOutput) { + return env.readCsvFile(verticesPath).types(Long.class) + .map( + new MapFunction, Long>() { + public Long map(Tuple1 value) { return value.f0; } + }); + } else { + return ConnectedComponentsData.getDefaultVertexDataSet(env); + } + } + + private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { + + if(fileOutput) { + return env.readCsvFile(edgesPath).fieldDelimiter(' ').types(Long.class, Long.class); + } else { + return ConnectedComponentsData.getDefaultEdgeDataSet(env); + } + } + + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/EnumTrianglesBasic.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java similarity index 55% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/EnumTrianglesBasic.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java index 4761b477d3054..6754f33a33d2e 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/EnumTrianglesBasic.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java @@ -12,7 +12,7 @@ * specific language governing permissions and limitations under the License. * **********************************************************************************************************************/ -package eu.stratosphere.example.java.triangles; +package eu.stratosphere.example.java.graph; import java.util.ArrayList; import java.util.Iterator; @@ -24,40 +24,104 @@ import eu.stratosphere.api.java.functions.GroupReduceFunction; import eu.stratosphere.api.java.functions.JoinFunction; import eu.stratosphere.api.java.functions.MapFunction; -import eu.stratosphere.example.java.triangles.util.EdgeData; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.Edge; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.Triad; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.TupleEdgeConverter; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.example.java.graph.util.EnumTrianglesData; +import eu.stratosphere.example.java.graph.util.EnumTrianglesDataTypes.Edge; +import eu.stratosphere.example.java.graph.util.EnumTrianglesDataTypes.Triad; import eu.stratosphere.util.Collector; /** * Triangle enumeration is a preprocessing step to find closely connected parts in graphs. * A triangle are three edges that connect three vertices with each other. * + *

* The algorithm works as follows: * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices * that are connected by two edges. Finally, all triads are filtered for which no third edge exists * that closes the triangle. * - * This implementation assumes that edges are represented as pairs of vertices and - * vertices are represented as Integer IDs. - * - * The lines of input files need to have the following format: - * ",\n" - * - * For example the input: - * "10,20\n10,30\n20,30\n" - * defines three edges (10,20), (10,30), (20,30) which build a triangle. + *

+ * Input files are plain text files must be formatted as follows: + *

    + *
  • Edges are represented as pairs for vertex IDs which are separated by space + * characters. Edges are separated by new-line characters.
    + * For example "1 2\n2 12\n1 12\n42 63\n" gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63) + * that include a triangle + *
+ *
+ *     (1)
+ *     /  \
+ *   (2)-(12)
+ * 
* + *

+ * This example shows how to use: + *

    + *
  • Custom Java objects which extend Tuple + *
  • Group Sorting + *
* */ +@SuppressWarnings("serial") public class EnumTrianglesBasic { - /** - * Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. - */ + static boolean fileOutput = false; + static String edgePath = null; + static String outputPath = null; + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + parseParameters(args); + + // set up execution environment + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // read input data + DataSet edges = getEdgeDataSet(env); + + // project edges by vertex id + DataSet edgesById = edges + .map(new EdgeByIdProjector()); + + DataSet triangles = edgesById + // build triads + .groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder()) + // filter triads + .join(edgesById).where(Triad.V2, Triad.V3).equalTo(Edge.V1, Edge.V2).with(new TriadFilter()); + + // emit result + if(fileOutput) { + triangles.writeAsCsv(outputPath, "\n", ","); + } else { + triangles.print(); + } + + // execute program + env.execute("Basic Triangle Enumeration Example"); + + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** Converts a Tuple2 into an Edge */ + public static class TupleEdgeConverter extends MapFunction, Edge> { + private final Edge outEdge = new Edge(); + + @Override + public Edge map(Tuple2 t) throws Exception { + outEdge.copyVerticesFromTuple2(t); + return outEdge; + } + } + + /** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */ private static class EdgeByIdProjector extends MapFunction { - private static final long serialVersionUID = 1L; @Override public Edge map(Edge inEdge) throws Exception { @@ -77,8 +141,6 @@ public Edge map(Edge inEdge) throws Exception { * Assumes that input edges share the first vertex and are in ascending order of the second vertex. */ private static class TriadBuilder extends GroupReduceFunction { - private static final long serialVersionUID = 1L; - private final List vertices = new ArrayList(); private final Triad outTriad = new Triad(); @@ -108,11 +170,8 @@ public void reduce(Iterator edges, Collector out) throws Exception } } - /** - * Filters triads (three vertices connected by two edges) without a closing third edge. - */ + /** Filters triads (three vertices connected by two edges) without a closing third edge. */ private static class TriadFilter extends JoinFunction { - private static final long serialVersionUID = 1L; @Override public Triad join(Triad triad, Edge edge) throws Exception { @@ -120,51 +179,39 @@ public Triad join(Triad triad, Edge edge) throws Exception { } } - public static void main(String[] args) throws Exception { - - String edgePath = "TESTDATA"; - String outPath = "STDOUT"; - - // parse input arguments - if(args.length > 0) { - edgePath = args[0]; - } - if(args.length > 1) { - outPath = args[1]; - } - - // set up execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* - // read input data - DataSet edges; - if(edgePath.equals("TESTDATA")) { - edges = EdgeData.getDefaultEdgeDataSet(env); + private static void parseParameters(String[] args) { + + if(args.length > 0) { + // parse input arguments + fileOutput = true; + if(args.length == 2) { + edgePath = args[0]; + outputPath = args[1]; + } else { + System.err.println("Usage: EnumTriangleBasic "); + System.exit(1); + } } else { - edges = env.readCsvFile(edgePath) - .fieldDelimiter(',') - .includeFields(true, true) - .types(Integer.class, Integer.class) - .map(new TupleEdgeConverter()); + System.out.println("Executing Enum Triangles Basic example with built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" Usage: EnumTriangleBasic "); } - - // project edges by vertex id - DataSet edgesById = edges - .map(new EdgeByIdProjector()); - - // build and filter triads - DataSet triangles = edgesById - .groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder()) - .join(edgesById).where(Triad.V2, Triad.V3).equalTo(Edge.V1, Edge.V2).with(new TriadFilter()); - - // emit triangles - if(outPath.equals("STDOUT")) { - triangles.print(); + } + + private static DataSet getEdgeDataSet(ExecutionEnvironment env) { + if(fileOutput) { + return env.readCsvFile(edgePath) + .fieldDelimiter(' ') + .includeFields(true, true) + .types(Integer.class, Integer.class) + .map(new TupleEdgeConverter()); } else { - triangles.writeAsCsv(outPath, "\n", ","); + return EnumTrianglesData.getDefaultEdgeDataSet(env); } - - // execute program - env.execute(); } + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/EnumTrianglesOpt.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java similarity index 68% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/EnumTrianglesOpt.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java index 9d67565cd4602..986fd9eaad8a0 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/EnumTrianglesOpt.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java @@ -12,7 +12,7 @@ * specific language governing permissions and limitations under the License. * **********************************************************************************************************************/ -package eu.stratosphere.example.java.triangles; +package eu.stratosphere.example.java.graph; import java.util.ArrayList; import java.util.Iterator; @@ -26,46 +26,117 @@ import eu.stratosphere.api.java.functions.JoinFunction; import eu.stratosphere.api.java.functions.MapFunction; import eu.stratosphere.api.java.functions.ReduceFunction; -import eu.stratosphere.example.java.triangles.util.EdgeData; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.Edge; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.EdgeWithDegrees; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.Triad; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.TupleEdgeConverter; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.example.java.graph.util.EnumTrianglesData; +import eu.stratosphere.example.java.graph.util.EnumTrianglesDataTypes.Edge; +import eu.stratosphere.example.java.graph.util.EnumTrianglesDataTypes.EdgeWithDegrees; +import eu.stratosphere.example.java.graph.util.EnumTrianglesDataTypes.Triad; import eu.stratosphere.util.Collector; /** * Triangle enumeration is a preprocessing step to find closely connected parts in graphs. * A triangle are three edges that connect three vertices with each other. * + *

* The basic algorithm works as follows: * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices * that are connected by two edges. Finally, all triads are filtered for which no third edge exists * that closes the triangle. * - * For a group of n edges that share a common vertex, the number of built triads is quadratic ((n*(n-1))/2). + *

+ * For a group of n edges that share a common vertex, the number of built triads is quadratic ((n*(n-1))/2). * Therefore, an optimization of the algorithm is to group edges on the vertex with the smaller output degree to * reduce the number of triads. * This implementation extends the basic algorithm by computing output degrees of edge vertices and * grouping on edges on the vertex with the smaller degree. - * - * This implementation assumes that edges are represented as pairs of vertices and - * vertices are represented as Integer IDs. * - * The lines of input files need to have the following format: - * ",\n" + *

+ * Input files are plain text files must be formatted as follows: + *

    + *
  • Edges are represented as pairs for vertex IDs which are separated by space + * characters. Edges are separated by new-line characters.
    + * For example "1 2\n2 12\n1 12\n42 63\n" gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63) + * that include a triangle + *
+ *
+ *     (1)
+ *     /  \
+ *   (2)-(12)
+ * 
* - * For example the input: - * "10,20\n10,30\n20,30\n" - * defines three edges (10,20), (10,30), (20,30) which build a triangle. + *

+ * This example shows how to use: + *

    + *
  • Custom Java objects which extend Tuple + *
  • Group Sorting + *
* */ +@SuppressWarnings("serial") public class EnumTrianglesOpt { - /** - * Emits for an edge the original edge and its switched version. - */ + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + parseParameters(args); + + // set up execution environment + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // read input data + DataSet edges = getEdgeDataSet(env); + + // annotate edges with degrees + DataSet edgesWithDegrees = edges + .flatMap(new EdgeDuplicator()) + .groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new DegreeCounter()) + .groupBy(EdgeWithDegrees.V1,EdgeWithDegrees.V2).reduce(new DegreeJoiner()); + + // project edges by degrees + DataSet edgesByDegree = edgesWithDegrees + .map(new EdgeByDegreeProjector()); + // project edges by vertex id + DataSet edgesById = edgesByDegree + .map(new EdgeByIdProjector()); + + DataSet triangles = edgesByDegree + // build triads + .groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder()) + // filter triads + .join(edgesById).where(Triad.V2,Triad.V3).equalTo(Edge.V1,Edge.V2).with(new TriadFilter()); + + // emit result + if(fileOutput) { + triangles.writeAsCsv(outputPath, "\n", ","); + } else { + triangles.print(); + } + + // execute program + env.execute("Triangle Enumeration Example"); + + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** Converts a Tuple2 into an Edge */ + public static class TupleEdgeConverter extends MapFunction, Edge> { + private final Edge outEdge = new Edge(); + + @Override + public Edge map(Tuple2 t) throws Exception { + outEdge.copyVerticesFromTuple2(t); + return outEdge; + } + } + + /** Emits for an edge the original edge and its switched version. */ private static class EdgeDuplicator extends FlatMapFunction { - private static final long serialVersionUID = 1L; @Override public void flatMap(Edge edge, Collector out) throws Exception { @@ -81,7 +152,6 @@ public void flatMap(Edge edge, Collector out) throws Exception { * For each emitted edge, the first vertex is the vertex with the smaller id. */ private static class DegreeCounter extends GroupReduceFunction { - private static final long serialVersionUID = 1L; final ArrayList otherVertices = new ArrayList(); final EdgeWithDegrees outputEdge = new EdgeWithDegrees(); @@ -130,7 +200,6 @@ public void reduce(Iterator edges, Collector out) throws * degree annotation. */ private static class DegreeJoiner extends ReduceFunction { - private static final long serialVersionUID = 1L; private final EdgeWithDegrees outEdge = new EdgeWithDegrees(); @Override @@ -149,11 +218,8 @@ public EdgeWithDegrees reduce(EdgeWithDegrees edge1, EdgeWithDegrees edge2) thro } } - /** - * Projects an edge (pair of vertices) such that the first vertex is the vertex with the smaller degree. - */ + /** Projects an edge (pair of vertices) such that the first vertex is the vertex with the smaller degree. */ private static class EdgeByDegreeProjector extends MapFunction { - private static final long serialVersionUID = 1L; private final Edge outEdge = new Edge(); @@ -173,11 +239,8 @@ public Edge map(EdgeWithDegrees inEdge) throws Exception { } } - /** - * Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. - */ + /** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */ private static class EdgeByIdProjector extends MapFunction { - private static final long serialVersionUID = 1L; @Override public Edge map(Edge inEdge) throws Exception { @@ -197,7 +260,6 @@ public Edge map(Edge inEdge) throws Exception { * Assumes that input edges share the first vertex and are in ascending order of the second vertex. */ private static class TriadBuilder extends GroupReduceFunction { - private static final long serialVersionUID = 1L; private final List vertices = new ArrayList(); private final Triad outTriad = new Triad(); @@ -228,11 +290,8 @@ public void reduce(Iterator edges, Collector out) throws Exception } } - /** - * Filters triads (three vertices connected by two edges) without a closing third edge. - */ + /** Filters triads (three vertices connected by two edges) without a closing third edge. */ private static class TriadFilter extends JoinFunction { - private static final long serialVersionUID = 1L; @Override public Triad join(Triad triad, Edge edge) throws Exception { @@ -240,60 +299,43 @@ public Triad join(Triad triad, Edge edge) throws Exception { } } - public static void main(String[] args) throws Exception { - - String edgePath = "TESTDATA"; - String outPath = "STDOUT"; + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static boolean fileOutput = false; + private static String edgePath = null; + private static String outputPath = null; + + private static void parseParameters(String[] args) { - // parse input arguments if(args.length > 0) { - edgePath = args[0]; - } - if(args.length > 1) { - outPath = args[1]; - } - - // set up execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // read input data - DataSet edges; - if(edgePath.equals("TESTDATA")) { - edges = EdgeData.getDefaultEdgeDataSet(env); + // parse input arguments + fileOutput = true; + if(args.length == 2) { + edgePath = args[0]; + outputPath = args[1]; + } else { + System.err.println("Usage: EnumTriangleBasic "); + System.exit(1); + } } else { - edges = env.readCsvFile(edgePath) - .fieldDelimiter(',') - .includeFields(true, true) - .types(Integer.class, Integer.class) - .map(new TupleEdgeConverter()); + System.out.println("Executing Enum Triangles Opt example with built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" Usage: EnumTriangleBasic "); } - - // annotate edges with degrees - DataSet edgesWithDegrees = edges - .flatMap(new EdgeDuplicator()) - .groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new DegreeCounter()) - .groupBy(EdgeWithDegrees.V1,EdgeWithDegrees.V2).reduce(new DegreeJoiner()); - - // project edges by degrees - DataSet edgesByDegree = edgesWithDegrees - .map(new EdgeByDegreeProjector()); - // project edges by vertex id - DataSet edgesById = edgesByDegree - .map(new EdgeByIdProjector()); - - // build and filter triads - DataSet triangles = edgesByDegree - .groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder()) - .join(edgesById).where(Triad.V2,Triad.V3).equalTo(Edge.V1,Edge.V2).with(new TriadFilter()); - - // emit triangles - if(outPath.equals("STDOUT")) { - triangles.print(); + } + + private static DataSet getEdgeDataSet(ExecutionEnvironment env) { + if(fileOutput) { + return env.readCsvFile(edgePath) + .fieldDelimiter(' ') + .includeFields(true, true) + .types(Integer.class, Integer.class) + .map(new TupleEdgeConverter()); } else { - triangles.writeAsCsv(outPath, "\n", ","); + return EnumTrianglesData.getDefaultEdgeDataSet(env); } - - // execute program - env.execute(); } + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java new file mode 100644 index 0000000000000..b3deb67abaa80 --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java @@ -0,0 +1,247 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.graph; + +import static eu.stratosphere.api.java.aggregation.Aggregations.SUM; + +import java.util.ArrayList; +import java.util.Iterator; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.IterativeDataSet; +import eu.stratosphere.api.java.functions.FilterFunction; +import eu.stratosphere.api.java.functions.FlatMapFunction; +import eu.stratosphere.api.java.functions.GroupReduceFunction; +import eu.stratosphere.api.java.functions.MapFunction; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.example.java.graph.util.PageRankData; +import eu.stratosphere.util.Collector; + +/** + * A basic implementation of the page rank algorithm using a bulk iteration. + * + *

+ * This implementation requires a set of pages (vertices) with associated ranks and a set + * of directed links (edges) as input and works as follows.
+ * In each iteration, the rank of every page is evenly distributed to all pages it points to. + * Each page collects the partial ranks of all pages that point to it, sums them up, and apply a dampening factor to the sum. + * The result is the new rank of the page. A new iteration is started with the new ranks of all pages. + * This implementation terminates after a fixed number of iterations. + * + *

+ * Input files are plain text files must be formatted as follows: + *

    + *
  • Pages represented as an (long) ID and a (double) rank separated by new-line characters.
    + * For example "1 0.4\n2 0.3\n12 0.15\n42 0.05\n63 0.1\n" gives five pages with associated ranks + * (1, 0.4), (2, 0.3), (12, 0.15), (42, 0.05), and (63, 0.1). Ranks should sum up to 1.0. + *
  • Page links are represented as pairs of page IDs which are separated by space + * characters. Edges are separated by new-line characters.
    + * For example "1 2\n2 12\n1 12\n42 63\n" gives four (directed) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63). + * For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself). + *
+ * + *

+ * This example shows how to use: + *

    + *
  • Bulk Iterations + *
  • Default Join + *
+ * + * + */ +@SuppressWarnings("serial") +public class PageRankBasic { + + private static final double DAMPENING_FACTOR = 0.85; + private static final double EPSILON = 0.0001; + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + parseParameters(args); + + // set up execution environment + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + // get input data + DataSet> pageWithRankInput = getPageWithRankDataSet(env); + DataSet> edgeInput = getEdgeDataSet(env); + + // build adjecency list from edge input + DataSet> adjacencyListInput = + edgeInput.groupBy(0).reduceGroup(new BuildOutgoingEdgeList()); + + // set iterative data set + IterativeDataSet> iteration = pageWithRankInput.iterate(maxIterations); + + DataSet> newRanks = iteration + // join pages with outgoing edges and distribute rank + .join(adjacencyListInput).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch()) + // collect and sum ranks + .groupBy(0).aggregate(SUM, 1) + // apply dampening factor + .map(new Dampener(numVertices)); + + DataSet> finalPageRanks = iteration.closeWith( + newRanks, + newRanks.join(iteration).where(0).equalTo(0) + // termination condition + .filter(new EpsilonFilter())); + + // emit result + if(fileOutput) { + finalPageRanks.writeAsCsv(outputPath, "\n", " "); + } else { + finalPageRanks.print(); + } + + // execute program + env.execute("Basic Page Rank Example"); + + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** + * A reduce function that takes a sequence of edges and builds the adjacency list for the vertex where the edges + * originate. Run as a preprocessing step. + */ + public static final class BuildOutgoingEdgeList extends GroupReduceFunction, Tuple2> { + + private final ArrayList neighbors = new ArrayList(); + + @Override + public void reduce(Iterator> values, Collector> out) { + neighbors.clear(); + Long id = 0L; + + while (values.hasNext()) { + Tuple2 n = values.next(); + id = n.f0; + neighbors.add(n.f1); + } + out.collect(new Tuple2(id, neighbors.toArray(new Long[neighbors.size()]))); + } + } + + /** + * Join function that distributes a fraction of a vertex's rank to all neighbors. + */ + public static final class JoinVertexWithEdgesMatch extends FlatMapFunction, Tuple2>, Tuple2> { + + @Override + public void flatMap(Tuple2, Tuple2> value, Collector> out){ + Long[] neigbors = value.f1.f1; + double rank = value.f0.f1; + double rankToDistribute = rank / ((double) neigbors.length); + + for (int i = 0; i < neigbors.length; i++) { + out.collect(new Tuple2(neigbors[i], rankToDistribute)); + } + } + } + + /** + * The function that applies the page rank dampening formula + */ + public static final class Dampener extends MapFunction, Tuple2> { + + private final double numVertices; + + public Dampener(double numVertices) { + this.numVertices = numVertices; + } + + @Override + public Tuple2 map(Tuple2 value) { + value.f1 = DAMPENING_FACTOR*value.f1 + (1-DAMPENING_FACTOR)/numVertices; + return value; + } + } + + /** + * Filter that filters vertices where the rank difference is below a threshold. + */ + public static final class EpsilonFilter extends FilterFunction, Tuple2>> { + + @Override + public boolean filter(Tuple2, Tuple2> value) { + return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON; + } + } + + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static boolean fileOutput = false; + private static String pageWithRankInputPath = null; + private static String edgeInputPath = null; + private static String outputPath = null; + private static int numVertices = 0; + private static int maxIterations = 10; + + private static void parseParameters(String[] args) { + + if(args.length > 0) { + if(args.length == 5) { + fileOutput = true; + pageWithRankInputPath = args[0]; + edgeInputPath = args[1]; + outputPath = args[2]; + numVertices = Integer.parseInt(args[3]); + maxIterations = Integer.parseInt(args[4]); + } else { + System.err.println("Usage: PageRankBasic "); + System.exit(1); + } + } else { + System.out.println("Executing PageRank Basic example with default parameters and built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" Usage: PageRankBasic "); + + numVertices = PageRankData.getNumberOfPages(); + } + } + + private static DataSet> getPageWithRankDataSet(ExecutionEnvironment env) { + if(fileOutput) { + return env.readCsvFile(pageWithRankInputPath) + .fieldDelimiter(' ') + .lineDelimiter("\n") + .types(Long.class, Double.class); + } else { + return PageRankData.getDefaultPageWithRankDataSet(env); + } + } + + private static DataSet> getEdgeDataSet(ExecutionEnvironment env) { + if(fileOutput) { + return env.readCsvFile(edgeInputPath) + .fieldDelimiter(' ') + .lineDelimiter("\n") + .types(Long.class, Long.class); + } else { + return PageRankData.getDefaultEdgeDataSet(env); + } + } + +} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/SimplePageRank.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/SimplePageRank.java deleted file mode 100644 index d9b56b4b6f74d..0000000000000 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/SimplePageRank.java +++ /dev/null @@ -1,154 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.example.java.graph; - -import static eu.stratosphere.api.java.aggregation.Aggregations.SUM; - -import java.util.ArrayList; -import java.util.Iterator; - -import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.api.java.IterativeDataSet; -import eu.stratosphere.api.java.functions.FilterFunction; -import eu.stratosphere.api.java.functions.FlatMapFunction; -import eu.stratosphere.api.java.functions.GroupReduceFunction; -import eu.stratosphere.api.java.functions.MapFunction; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.util.Collector; - -@SuppressWarnings("serial") -public class SimplePageRank { - - private static final double DAMPENING_FACTOR = 0.85; - - private static final double EPSILON = 0.0001; - - - public static void main(String[] args) throws Exception { - - if (args.length < 5) { - System.err.println("Usage: SimpePageRank "); - return; - } - - final String pageWithRankInputPath = args[0]; - final String adjacencyListInputPath = args[1]; - final String outputPath = args[2]; - final int numVertices = Integer.parseInt(args[3]); - final int maxIterations = Integer.parseInt(args[4]); - - runPageRank(pageWithRankInputPath, adjacencyListInputPath, outputPath, numVertices, maxIterations); - - } - - public static void runPageRank(String verticesPath, String edgesPath, String outputPath, int numVertices, int maxIterations) throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // vertices: vertexID, initialRank - DataSet> pageWithRankInput = env.readCsvFile(verticesPath).types(Long.class, Double.class); - - // edge adjacency list: vertexID, vertexID[] - DataSet> adjacencyListInput = env.readCsvFile(edgesPath).types(Long.class, Long.class) - .groupBy(0).reduceGroup(new OutgoingEdgeCounter()); - - IterativeDataSet> iteration = pageWithRankInput.iterate(maxIterations); - - DataSet> newRanks = iteration - .join(adjacencyListInput).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch()) - .groupBy(0).aggregate(SUM, 1) - .map(new Dampener(numVertices)); - - iteration.closeWith(newRanks, - newRanks.join(iteration).where(0).equalTo(0).filter(new EpsilonFilter())) // termination condition - .writeAsCsv(outputPath); - - env.execute(); - } - - - // -------------------------------------------------------------------------------------------- - // The user-defined functions for parts of page rank - // -------------------------------------------------------------------------------------------- - - /** - * A reduce function that takes a sequence of edges and builds the adjacency list for the vertex where the edges - * originate. Run as a preprocessing step. - */ - public static final class OutgoingEdgeCounter extends GroupReduceFunction, Tuple2> { - - private final ArrayList neighbors = new ArrayList(); - - @Override - public void reduce(Iterator> values, Collector> out) { - neighbors.clear(); - Long id = 0L; - - while (values.hasNext()) { - Tuple2 n = values.next(); - id = n.f0; - neighbors.add(n.f1); - } - out.collect(new Tuple2(id, neighbors.toArray(new Long[neighbors.size()]))); - } - } - - /** - * Join function that distributes a fraction of a vertex's rank to all neighbors. - */ - public static final class JoinVertexWithEdgesMatch extends FlatMapFunction, Tuple2>, Tuple2> { - - @Override - public void flatMap(Tuple2, Tuple2> value, Collector> out){ - Long[] neigbors = value.f1.f1; - double rank = value.f0.f1; - double rankToDistribute = rank / ((double) neigbors.length); - - for (int i = 0; i < neigbors.length; i++) { - out.collect(new Tuple2(neigbors[i], rankToDistribute)); - } - } - } - - /** - * The function that applies the page rank dampening formula - */ - public static final class Dampener extends MapFunction, Tuple2> { - - private final double numVertices; - - public Dampener(double numVertices) { - this.numVertices = numVertices; - } - - @Override - public Tuple2 map(Tuple2 value) { - value.f1 = DAMPENING_FACTOR*value.f1 + (1-DAMPENING_FACTOR)/numVertices; - return value; - } - } - - /** - * Filter that filters vertices where the rank difference is below a threshold. - */ - public static final class EpsilonFilter extends FilterFunction, Tuple2>> { - - @Override - public boolean filter(Tuple2, Tuple2> value) { - return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON; - } - } -} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/ConnectedComponentsData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/ConnectedComponentsData.java new file mode 100644 index 0000000000000..c9b8edaa2466a --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/ConnectedComponentsData.java @@ -0,0 +1,53 @@ +/*********************************************************************************************************************** + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.graph.util; + +import java.util.ArrayList; +import java.util.List; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.tuple.Tuple2; + +public class ConnectedComponentsData { + + public static DataSet getDefaultVertexDataSet(ExecutionEnvironment env) { + + return env.fromElements( + 1L, 2L, 3L, 4L, 5L, + 6L, 7L, 8L, 9L, 10L, + 11L, 12L, 13L, 14L, 15L); + } + + public static DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(); + data.add(new Tuple2(1L, 2L)); + data.add(new Tuple2(2L, 3L)); + data.add(new Tuple2(2L, 4L)); + data.add(new Tuple2(3L, 5L)); + data.add(new Tuple2(6L, 7L)); + data.add(new Tuple2(8L, 9L)); + data.add(new Tuple2(8L, 10L)); + data.add(new Tuple2(5L, 11L)); + data.add(new Tuple2(11L, 12L)); + data.add(new Tuple2(10L, 13L)); + data.add(new Tuple2(9L, 14L)); + data.add(new Tuple2(13L, 14L)); + data.add(new Tuple2(1L, 15L)); + + return env.fromCollection(data); + } + +} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/util/EdgeData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesData.java similarity index 89% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/util/EdgeData.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesData.java index 5b1ee293ecf40..d12279657cf9e 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/util/EdgeData.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesData.java @@ -12,13 +12,14 @@ * specific language governing permissions and limitations under the License. * **********************************************************************************************************************/ -package eu.stratosphere.example.java.triangles.util; + +package eu.stratosphere.example.java.graph.util; import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.example.java.triangles.util.EdgeDataTypes.Edge; +import eu.stratosphere.example.java.graph.util.EnumTrianglesDataTypes.Edge; -public class EdgeData { +public class EnumTrianglesData { public static DataSet getDefaultEdgeDataSet(ExecutionEnvironment env) { diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/util/EdgeDataTypes.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java similarity index 88% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/util/EdgeDataTypes.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java index 4603989414c46..39719857b5b2c 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/triangles/util/EdgeDataTypes.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java @@ -12,14 +12,14 @@ * specific language governing permissions and limitations under the License. * **********************************************************************************************************************/ -package eu.stratosphere.example.java.triangles.util; -import eu.stratosphere.api.java.functions.MapFunction; +package eu.stratosphere.example.java.graph.util; + import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; import eu.stratosphere.api.java.tuple.Tuple4; -public class EdgeDataTypes { +public class EnumTrianglesDataTypes { public static class Edge extends Tuple2 { private static final long serialVersionUID = 1L; @@ -110,17 +110,4 @@ public void copyFrom(final EdgeWithDegrees edge) { } - public static class TupleEdgeConverter extends MapFunction, Edge> { - private static final long serialVersionUID = 1L; - - private final Edge outEdge = new Edge(); - - @Override - public Edge map(Tuple2 t) throws Exception { - outEdge.copyVerticesFromTuple2(t); - return outEdge; - } - - } - } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/PageRankData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/PageRankData.java new file mode 100644 index 0000000000000..280f6ecbdb3b9 --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/PageRankData.java @@ -0,0 +1,81 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.graph.util; + +import java.util.ArrayList; +import java.util.List; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.tuple.Tuple2; + +public class PageRankData { + + private static int numPages = 15; + + public static DataSet> getDefaultPageWithRankDataSet(ExecutionEnvironment env) { + + double initRank = 1.0 / numPages; + + List> data = new ArrayList>(); + + for(int i=0; i(i+1L, initRank)); + } + return env.fromCollection(data); + } + + public static DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(); + data.add(new Tuple2(1L, 2L)); + data.add(new Tuple2(1L, 15L)); + data.add(new Tuple2(2L, 3L)); + data.add(new Tuple2(2L, 4L)); + data.add(new Tuple2(2L, 5L)); + data.add(new Tuple2(2L, 6L)); + data.add(new Tuple2(2L, 7L)); + data.add(new Tuple2(3L, 13L)); + data.add(new Tuple2(4L, 2L)); + data.add(new Tuple2(5L, 11L)); + data.add(new Tuple2(5L, 12L)); + data.add(new Tuple2(6L, 1L)); + data.add(new Tuple2(6L, 7L)); + data.add(new Tuple2(6L, 8L)); + data.add(new Tuple2(7L, 1L)); + data.add(new Tuple2(7L, 8L)); + data.add(new Tuple2(8L, 1L)); + data.add(new Tuple2(8L, 9L)); + data.add(new Tuple2(8L, 10L)); + data.add(new Tuple2(9L, 14L)); + data.add(new Tuple2(9L, 1L)); + data.add(new Tuple2(10L, 1L)); + data.add(new Tuple2(10L, 13L)); + data.add(new Tuple2(11L, 12L)); + data.add(new Tuple2(11L, 1L)); + data.add(new Tuple2(12L, 1L)); + data.add(new Tuple2(13L, 14L)); + data.add(new Tuple2(14L, 12L)); + data.add(new Tuple2(15L, 1L)); + + return env.fromCollection(data); + } + + public static int getNumberOfPages() { + return numPages; + } + +} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/incremental/pagerank/SimpleDeltaPageRank.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/incremental/pagerank/SimpleDeltaPageRank.java deleted file mode 100644 index 87b6474a8dda1..0000000000000 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/incremental/pagerank/SimpleDeltaPageRank.java +++ /dev/null @@ -1,157 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.example.java.incremental.pagerank; - - -import java.util.Iterator; - -import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.DeltaIteration; -import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.api.java.functions.GroupReduceFunction; -import eu.stratosphere.api.java.functions.JoinFunction; -import eu.stratosphere.api.java.functions.MapFunction; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.api.java.tuple.Tuple3; -import eu.stratosphere.util.Collector; - -public class SimpleDeltaPageRank { - - public static final class RankComparisonMatch extends JoinFunction, Tuple2, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 join(Tuple2 vertexWithDelta, - Tuple2 vertexWithOldRank) throws Exception { - - return new Tuple2(vertexWithOldRank.f0, vertexWithDelta.f1 + vertexWithOldRank.f1); - } - } - -// public static final class UpdateRankReduceDelta extends ReduceFunction> { -// -// private static final long serialVersionUID = 1L; -// -// @Override -// public Tuple2 reduce(Tuple2 value1, -// Tuple2 value2) throws Exception { -// -// double rankSum = value1.f1 + value2.f1; -// -// // ignore small deltas -// if (Math.abs(rankSum) > 0.00001) { -// return new Tuple2(value1.f0, rankSum); -// } -// return null; -// } -// } - - public static final class UpdateRankReduceDelta extends GroupReduceFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterator> values, - Collector> out) throws Exception { - - double rankSum = 0.0; - Tuple2 currentTuple = null; - - while (values.hasNext()) { - currentTuple = values.next(); - rankSum += currentTuple.f1; - } - - // ignore small deltas - if (Math.abs(rankSum) > 0.00001) { - out.collect(new Tuple2(currentTuple.f0, rankSum)); - } - } - } - - - public static final class PRDependenciesComputationMatchDelta extends JoinFunction, Tuple3, Tuple2> { - - private static final long serialVersionUID = 1L; - - /* - * (vId, rank) x (srcId, trgId, weight) => (trgId, rank / weight) - */ - @Override - public Tuple2 join(Tuple2 vertexWithRank, - Tuple3 edgeWithWeight) throws Exception { - - return new Tuple2(edgeWithWeight.f1, (Double) (vertexWithRank.f1 / edgeWithWeight.f2)); - } - } - - public static final class Mapper extends MapFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 map(Tuple2 value) - throws Exception { - - return value; - } - - } - - public static void main(String[] args) throws Exception { - - if (args.length < 5) { - System.err.println("Usage: SimpePageRank "); - return; - } - - final int dop = Integer.parseInt(args[0]); - final String solutionSetInputPath = args[1]; - final String deltasInputPath = args[2]; - final String dependencySetInputPath = args[3]; - final String outputPath = args[4]; - final int maxIterations = Integer.parseInt(args[5]); - - run(dop, solutionSetInputPath, deltasInputPath, dependencySetInputPath, outputPath, maxIterations, true); - - } - - public static void run(int dop, String solutionSetInputPath, String deltasInputPath, String dependencySetInputPath, String outputPath, int maxIterations, boolean terminationCriterion) throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(dop); - - DataSet> initialSolutionSet = env.readCsvFile(solutionSetInputPath).fieldDelimiter(' ').types(Long.class, Double.class).map(new Mapper()); - - DataSet> initialDeltaSet = env.readCsvFile(deltasInputPath).fieldDelimiter(' ').types(Long.class, Double.class); - - DataSet> dependencySetInput = env.readCsvFile(dependencySetInputPath).fieldDelimiter(' ').types(Long.class, Long.class, Long.class); - - int keyPosition = 0; - DeltaIteration, Tuple2> iteration = initialSolutionSet.iterateDelta(initialDeltaSet, maxIterations, keyPosition); - - DataSet> updateRanks = iteration.getWorkset().join(dependencySetInput).where(0).equalTo(0).with(new PRDependenciesComputationMatchDelta()) - .groupBy(0).reduceGroup(new UpdateRankReduceDelta()); - - DataSet> oldRankComparison = updateRanks.join(iteration.getSolutionSet()).where(0).equalTo(0).with(new RankComparisonMatch()); - - iteration.closeWith(oldRankComparison, updateRanks).writeAsCsv(outputPath); - - env.execute(); - - } - -} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery10.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery10.java index 267f17f557b43..6b1405163d7e6 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery10.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery10.java @@ -16,10 +16,9 @@ import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.aggregation.Aggregations; import eu.stratosphere.api.java.functions.FilterFunction; -import eu.stratosphere.api.java.functions.JoinFunction; import eu.stratosphere.api.java.functions.MapFunction; -import eu.stratosphere.api.java.functions.ReduceFunction; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; import eu.stratosphere.api.java.tuple.Tuple4; @@ -28,19 +27,21 @@ /** * This program implements a modified version of the TPC-H query 10. - * * The original query can be found at - * http://www.tpc.org/tpch/spec/tpch2.16.0.pdf (page 45). + * http://www.tpc.org/tpch/spec/tpch2.16.0.pdf (page 45). * + *

* This program implements the following SQL equivalent: * + *

+ *

  * SELECT 
- *        c_custkey, 
+ *        c_custkey,
  *        c_name, 
  *        c_address,
  *        n_name, 
  *        c_acctbal
- *        sum(l_extendedprice * (1 - l_discount)) as revenue,  
+ *        SUM(l_extendedprice * (1 - l_discount)) AS revenue,  
  * FROM   
  *        customer, 
  *        orders, 
@@ -58,144 +59,178 @@
  *        c_acctbal, 
  *        n_name, 
  *        c_address
+ * 
* + *

* Compared to the original TPC-H query this version does not print * c_phone and c_comment, only filters by years greater than 1990 instead of * a period of 3 months, and does not sort the result by revenue. + * + *

+ * Input files are plain text CSV files using the pipe character ('|') as field separator + * as generated by the TPC-H data generator which is available at http://www.tpc.org/tpch/. + * + *

+ * This example shows how to use: + *

    + *
  • tuple data types + *
  • inline-defined functions + *
  • projection and join projection + *
  • build-in aggregation functions + *
*/ - @SuppressWarnings("serial") public class TPCHQuery10 { + // ************************************************************************* + // PROGRAM + // ************************************************************************* + public static void main(String[] args) throws Exception { - if (args.length < 5) { - System.err.println("Parameters: "); - return; - } - - final String customerPath = args[0]; - final String ordersPath = args[1]; - final String lineitemPath = args[2]; - final String nationPath = args[3]; - final String outPath = args[4]; + parseParameters(args); final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - // read in customer table file - // customer: custkey, name, address, nationkey, acctbal - DataSet> customers = env.readCsvFile(customerPath).fieldDelimiter('|') - .includeFields("11110100").types(Integer.class, String.class, String.class, Integer.class, Double.class); + // get customer data set: (custkey, name, address, nationkey, acctbal) + DataSet> customers = getCustomerDataSet(env); - // read in orders table file - // order: orderkey, custkey, orderdate - DataSet> orders = env.readCsvFile(ordersPath).fieldDelimiter('|').includeFields("110010000") - .types(Integer.class, Integer.class, String.class); + // get orders data set: (orderkey, custkey, orderdate) + DataSet> orders = getOrdersDataSet(env); - // read in lineitem table file - // lineitem: orderkey, extendedprice, discount, returnflag - DataSet> lineitems = env.readCsvFile(lineitemPath).fieldDelimiter('|') - .includeFields("1000011010000000").types(Integer.class, Double.class, Double.class, String.class); + // get lineitem data set: (orderkey, extendedprice, discount, returnflag) + DataSet> lineitems = getLineitemDataSet(env); - // read in nation table file - // nation: nationkey, name - DataSet> nations = env.readCsvFile(nationPath).fieldDelimiter('|').includeFields("1100") - .types(Integer.class, String.class); + // get nation data set: (nationkey, name) + DataSet> nations = getNationsDataSet(env); - // orders filtered by year: orderkey, custkey - DataSet> ordersFilteredByYear = orders + // orders filtered by year: (orderkey, custkey) + DataSet> ordersFilteredByYear = // filter by year - .filter(new FilterFunction>() { - @Override - public boolean filter(Tuple3 t) { - int year = Integer.parseInt(t.f2.substring(0, 4)); - return year > 1990; - } - }) - // remove date as it is not necessary anymore - .map(new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple3 t) { - return new Tuple2(t.f0, t.f1); - } - }); + orders.filter( + new FilterFunction>() { + @Override + public boolean filter(Tuple3 t) { + int year = Integer.parseInt(t.f2.substring(0, 4)); + return year > 1990; + } + }) + // project fields out that are no longer required + .project(0,1).types(Integer.class, Integer.class); - // lineitems filtered by flag: orderkey, extendedprice, discount - DataSet> lineitemsFilteredByFlag = lineitems + // lineitems filtered by flag: (orderkey, extendedprice, discount) + DataSet> lineitemsFilteredByFlag = // filter by flag - .filter(new FilterFunction>() { - @Override - public boolean filter(Tuple4 t) - throws Exception { - return t.f3.equals("R"); - } - }) - // remove flag as it is not necessary anymore - .map(new MapFunction, Tuple3>() { - @Override - public Tuple3 map(Tuple4 t) { - return new Tuple3(t.f0, t.f1, t.f2); - } - }); + lineitems.filter(new FilterFunction>() { + @Override + public boolean filter(Tuple4 t) + throws Exception { + return t.f3.equals("R"); + } + }) + // project fields out that are no longer required + .project(0,1,2).types(Integer.class, Double.class, Double.class); - // join orders with lineitems - // custkey, extendedprice, discount - DataSet> lineitemsOfCustomerKey = ordersFilteredByYear.joinWithHuge(lineitemsFilteredByFlag) - .where(0).equalTo(0) - .with(new JoinFunction, Tuple3, Tuple3>() { - @Override - public Tuple3 join(Tuple2 o, Tuple3 l) { - return new Tuple3(o.f1, l.f1, l.f2); - } - }); + // join orders with lineitems: (custkey, extendedprice, discount) + DataSet> lineitemsOfCustomerKey = + ordersFilteredByYear.joinWithHuge(lineitemsFilteredByFlag) + .where(0).equalTo(0) + .projectFirst(1).projectSecond(1,2) + .types(Integer.class, Double.class, Double.class); - // aggregate for revenue - // custkey, revenue + // aggregate for revenue: (custkey, revenue) DataSet> revenueOfCustomerKey = lineitemsOfCustomerKey // calculate the revenue for each item .map(new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple3 t) { - // revenue per item = l_extendedprice * (1 - l_discount) - return new Tuple2(t.f0, t.f1 * (1 - t.f2)); - } - }) + @Override + public Tuple2 map(Tuple3 t) { + // revenue per item = l_extendedprice * (1 - l_discount) + return new Tuple2(t.f0, t.f1 * (1 - t.f2)); + } + }) // aggregate the revenues per item to revenue per customer - .groupBy(0).reduce(new ReduceFunction>() { - @Override - public Tuple2 reduce(Tuple2 t1, Tuple2 t2) { - return new Tuple2(t1.f0, t1.f1+t2.f1); - } - }); + .groupBy(0).aggregate(Aggregations.SUM, 1); - // join customer with nation - // custkey, name, address, nationname, acctbal + // join customer with nation (custkey, name, address, nationname, acctbal) DataSet> customerWithNation = customers - .joinWithTiny(nations) - .where(3) - .equalTo(0) - .with(new JoinFunction, Tuple2, Tuple5>() { - @Override - public Tuple5 join(Tuple5 c, Tuple2 n) { - return new Tuple5(c.f0, c.f1, c.f2, n.f1, c.f4); - } - }); + .joinWithTiny(nations) + .where(3).equalTo(0) + .projectFirst(0,1,2).projectSecond(1).projectFirst(4) + .types(Integer.class, String.class, String.class, String.class, Double.class); + + // join customer (with nation) with revenue (custkey, name, address, nationname, acctbal, revenue) + DataSet> customerWithRevenue = + customerWithNation.join(revenueOfCustomerKey) + .where(0).equalTo(0) + .projectFirst(0,1,2,3,4).projectSecond(1) + .types(Integer.class, String.class, String.class, String.class, Double.class, Double.class); - // join customer (with nation) with revenue - // custkey, name, address, nationname, acctbal, revenue - DataSet> customerWithRevenue = customerWithNation - .join(revenueOfCustomerKey) - .where(0) - .equalTo(0) - .with(new JoinFunction, Tuple2, Tuple6>() { - @Override - public Tuple6 join(Tuple5 c, Tuple2 r) { - return new Tuple6(c.f0, c.f1, c.f2, c.f3, c.f4, r.f1); - } - }); + // emit result + customerWithRevenue.writeAsCsv(outputPath); + + // execute program + env.execute("TPCH Query 10 Example"); + + } + + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static String customerPath; + private static String ordersPath; + private static String lineitemPath; + private static String nationPath; + private static String outputPath; + + private static void parseParameters(String[] programArguments) { + + if(programArguments.length > 0) { + if(programArguments.length == 5) { + customerPath = programArguments[0]; + ordersPath = programArguments[1]; + lineitemPath = programArguments[2]; + nationPath = programArguments[3]; + outputPath = programArguments[4]; + } else { + System.err.println("Usage: TPCHQuery10 "); + System.exit(1); + } + } else { + System.err.println("This program expects data from the TPC-H benchmark as input data.\n" + + " Due to legal restrictions, we can not ship generated data.\n" + + " You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + + " Usage: TPCHQuery10 "); + System.exit(1); + } + } + + private static DataSet> getCustomerDataSet(ExecutionEnvironment env) { + return env.readCsvFile(customerPath) + .fieldDelimiter('|') + .includeFields("11110100") + .types(Integer.class, String.class, String.class, Integer.class, Double.class); + } + + private static DataSet> getOrdersDataSet(ExecutionEnvironment env) { + return env.readCsvFile(ordersPath) + .fieldDelimiter('|') + .includeFields("110010000") + .types(Integer.class, Integer.class, String.class); + } - // write the result and execute - customerWithRevenue.writeAsCsv(outPath); - env.execute(); + private static DataSet> getLineitemDataSet(ExecutionEnvironment env) { + return env.readCsvFile(lineitemPath) + .fieldDelimiter('|') + .includeFields("1000011010000000") + .types(Integer.class, Double.class, Double.class, String.class); + } + + private static DataSet> getNationsDataSet(ExecutionEnvironment env) { + return env.readCsvFile(nationPath) + .fieldDelimiter('|') + .includeFields("1100") + .types(Integer.class, String.class); } + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java index 2acd91c18430c..794eb2d4c2feb 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java @@ -23,100 +23,196 @@ import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.aggregation.Aggregations; import eu.stratosphere.api.java.functions.FilterFunction; import eu.stratosphere.api.java.functions.JoinFunction; -import eu.stratosphere.api.java.functions.ReduceFunction; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; import eu.stratosphere.api.java.tuple.Tuple4; import eu.stratosphere.api.java.tuple.Tuple5; /** - * This program implements a modified version of the TPC-H query 3. The - * example demonstrates how to assign names to fields by extending the Tuple class. + * This program implements a modified version of the TPC-H query 3. The + * example demonstrates how to assign names to fields by extending the Tuple class. + * The original query can be found at + * http://www.tpc.org/tpch/spec/tpch2.16.0.pdf (page 29). * - * The original query can be found at - * http://www.tpc.org/tpch/spec/tpch2.16.0.pdf (page 29). + *

+ * This program implements the following SQL equivalent: * - * This program implements the following SQL equivalent: + *

+ *

+ * SELECT 
+ *      l_orderkey, 
+ *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
+ *      o_orderdate, 
+ *      o_shippriority 
+ * FROM customer, 
+ *      orders, 
+ *      lineitem 
+ * WHERE
+ *      c_mktsegment = '[SEGMENT]' 
+ *      AND c_custkey = o_custkey
+ *      AND l_orderkey = o_orderkey
+ *      AND o_orderdate < date '[DATE]'
+ *      AND l_shipdate > date '[DATE]'
+ * GROUP BY
+ *      l_orderkey, 
+ *      o_orderdate, 
+ *      o_shippriority;
+ * 
* - * select l_orderkey, - * sum(l_extendedprice*(1-l_discount)) as revenue, - * o_orderdate, - * o_shippriority from customer, - * orders, - * lineitem - * where - * c_mktsegment = '[SEGMENT]' and - * c_custkey = o_custkey and - * l_orderkey = o_orderkey and - * o_orderdate < date '[DATE]' and - * l_shipdate > date '[DATE]' - * group by - * l_orderkey, - * o_orderdate, - * o_shippriority - * order by //not yet - * revenue desc, - * o_orderdate; + *

+ * Compared to the original TPC-H query this version does not sort the result by revenue + * and orderdate. * + *

+ * Input files are plain text CSV files using the pipe character ('|') as field separator + * as generated by the TPC-H data generator which is available at http://www.tpc.org/tpch/. + * + *

+ * This example shows how to use: + *

    + *
  • custom data type derived from tuple data types + *
  • inline-defined functions + *
  • build-in aggregation functions + *
*/ @SuppressWarnings("serial") public class TPCHQuery3 { - // -------------------------------------------------------------------------------------------- - // Custom type classes - // -------------------------------------------------------------------------------------------- + // ************************************************************************* + // PROGRAM + // ************************************************************************* - public static class Lineitem extends Tuple4 { + public static void main(String[] args) throws Exception { + + parseParameters(args); - public Integer getOrderkey() { - return this.f0; - } + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - public Double getDiscount() { - return this.f2; - } + // get input data + DataSet li = getLineitemDataSet(env); + DataSet or = getOrdersDataSet(env); + DataSet cust = getCustomerDataSet(env); + + // Filter market segment "AUTOMOBILE" + cust = cust.filter( + new FilterFunction() { + @Override + public boolean filter(Customer value) { + return value.getMktsegment().equals("AUTOMOBILE"); + } + }); - public Double getExtendedprice() { - return this.f1; - } + // Filter all Orders with o_orderdate < 12.03.1995 + or = or.filter( + new FilterFunction() { + private DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + private Date date; + + { + Calendar cal = Calendar.getInstance(); + cal.set(1995, 3, 12); + date = cal.getTime(); + } + + @Override + public boolean filter(Order value) throws ParseException { + Date orderDate = format.parse(value.getOrderdate()); + return orderDate.before(date); + } + }); + + // Filter all Lineitems with l_shipdate > 12.03.1995 + li = li.filter( + new FilterFunction() { + private DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + private Date date; + + { + Calendar cal = Calendar.getInstance(); + cal.set(1995, 3, 12); + date = cal.getTime(); + } + + @Override + public boolean filter(Lineitem value) throws ParseException { + Date shipDate = format.parse(value.getShipdate()); + return shipDate.after(date); + } + }); - public String getShipdate() { - return this.f3; - } + // Join customers with orders and package them into a ShippingPriorityItem + DataSet customerWithOrders = + cust.join(or) + .where(0) + .equalTo(0) + .with( + new JoinFunction() { + @Override + public ShippingPriorityItem join(Customer first, Order second) { + return new ShippingPriorityItem(0, 0.0, second.getOrderdate(), + second.getShippriority(), second.getOrderkey()); + } + }); + + // Join the last join result with Orders + DataSet joined = + customerWithOrders.join(li) + .where(4) + .equalTo(0) + .with( + new JoinFunction() { + @Override + public ShippingPriorityItem join(ShippingPriorityItem first, Lineitem second) { + first.setL_Orderkey(second.getOrderkey()); + first.setRevenue(second.getExtendedprice() * (1 - second.getDiscount())); + return first; + } + }); + + // Group by l_orderkey, o_orderdate and o_shippriority and compute revenue sum + joined = joined + .groupBy(0, 2, 3) + .aggregate(Aggregations.SUM, 1); + + // emit result + joined.writeAsCsv(outputPath, "\n", "|"); + + // execute program + env.execute("TPCH Query 3 Example"); + + } + + // ************************************************************************* + // DATA TYPES + // ************************************************************************* + + public static class Lineitem extends Tuple4 { + + public Integer getOrderkey() { return this.f0; } + public Double getDiscount() { return this.f2; } + public Double getExtendedprice() { return this.f1; } + public String getShipdate() { return this.f3; } } public static class Customer extends Tuple2 { - public Integer getCustKey() { - return this.f0; - } - - public String getMktsegment() { - return this.f1; - } + public Integer getCustKey() { return this.f0; } + public String getMktsegment() { return this.f1; } } public static class Order extends Tuple3 { - public Integer getOrderkey() { - return this.f0; - } - - public String getOrderdate() { - return this.f1; - } - - public Integer getShippriority() { - return this.f2; - } + public Integer getOrderkey() { return this.f0; } + public String getOrderdate() { return this.f1; } + public Integer getShippriority() { return this.f2; } } public static class ShippingPriorityItem extends Tuple5 { - public ShippingPriorityItem() { - } + public ShippingPriorityItem() { } public ShippingPriorityItem(Integer l_orderkey, Double revenue, String o_orderdate, Integer o_shippriority, Integer o_orderkey) { @@ -127,175 +223,65 @@ public ShippingPriorityItem(Integer l_orderkey, Double revenue, this.f4 = o_orderkey; } - public Integer getL_Orderkey() { - return this.f0; - } - - public void setL_Orderkey(Integer l_orderkey) { - this.f0 = l_orderkey; - } - - public Double getRevenue() { - return this.f1; - } - - public void setRevenue(Double revenue) { - this.f1 = revenue; - } - - public String getOrderdate() { - return this.f2; - } - - public Integer getShippriority() { - return this.f3; - } - - public Integer getO_Orderkey() { - return this.f4; - } + public Integer getL_Orderkey() { return this.f0; } + public void setL_Orderkey(Integer l_orderkey) { this.f0 = l_orderkey; } + public Double getRevenue() { return this.f1; } + public void setRevenue(Double revenue) { this.f1 = revenue; } + + public String getOrderdate() { return this.f2; } + public Integer getShippriority() { return this.f3; } + public Integer getO_Orderkey() { return this.f4; } } + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* - // -------------------------------------------------------------------------------------------- - // Query program - // -------------------------------------------------------------------------------------------- + private static String lineitemPath; + private static String customerPath; + private static String ordersPath; + private static String outputPath; - /* - * This example TPCH3 query uses custom objects. - */ - public static void main(String[] args) throws Exception { - if (args.length < 3) { - System.err.println("Parameters: []."); - return; - } - - final String lineitemPath = args[0]; - final String customerPath = args[1]; - final String ordersPath = args[2]; - final String resultPath = args.length >= 4 ? args[4] : null; - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - /* - * Read Data from files - */ - DataSet li = env - .readCsvFile(lineitemPath).fieldDelimiter('|') - .includeFields("1000011000100000") - .tupleType(Lineitem.class); - - DataSet or = env - .readCsvFile(ordersPath).fieldDelimiter('|') - .includeFields("100010010") - .tupleType(Order.class); - - DataSet cust = env - .readCsvFile(customerPath).fieldDelimiter('|') - .includeFields("10000010") - .tupleType(Customer.class); + private static void parseParameters(String[] programArguments) { - /* - * Filter market segment "AUTOMOBILE" - */ - cust = cust.filter(new FilterFunction() { - @Override - public boolean filter(Customer value) { - return value.getMktsegment().equals("AUTOMOBILE"); - } - }); - - /* - * Filter all Orders with o_orderdate < 12.03.1995 - */ - or = or.filter(new FilterFunction() { - private DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); - private Date date; - - { - Calendar cal = Calendar.getInstance(); - cal.set(1995, 3, 12); - date = cal.getTime(); + if(programArguments.length > 0) { + if(programArguments.length == 4) { + lineitemPath = programArguments[0]; + customerPath = programArguments[1]; + ordersPath = programArguments[2]; + outputPath = programArguments[3]; + } else { + System.err.println("Usage: TPCHQuery3 "); + System.exit(1); } - - @Override - public boolean filter(Order value) throws ParseException { - Date orderDate = format.parse(value.getOrderdate()); - return orderDate.before(date); - } - }); - - /* - * Filter all Lineitems with l_shipdate > 12.03.1995 - */ - li = li.filter(new FilterFunction() { - private DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); - private Date date; - - { - Calendar cal = Calendar.getInstance(); - cal.set(1995, 3, 12); - date = cal.getTime(); - } - - @Override - public boolean filter(Lineitem value) throws ParseException { - Date shipDate = format.parse(value.getShipdate()); - return shipDate.after(date); - } - }); - - /* - * Join customers with orders and package them into a ShippingPriorityItem - */ - DataSet customerWithOrders = cust - .join(or) - .where(0) - .equalTo(0) - .with(new JoinFunction() { - @Override - public ShippingPriorityItem join(Customer first, Order second) { - return new ShippingPriorityItem(0, 0.0, second.getOrderdate(), - second.getShippriority(), second.getOrderkey()); - } - }); - - /* - * Join the last join result with Orders - */ - DataSet joined = customerWithOrders - .join(li) - .where(4) - .equalTo(0) - .with(new JoinFunction() { - @Override - public ShippingPriorityItem join(ShippingPriorityItem first, Lineitem second) { - first.setL_Orderkey(second.getOrderkey()); - first.setRevenue(second.getExtendedprice() * (1 - second.getDiscount())); - return first; - } - }); - - /* - * GroupBy l_orderkey, o_orderdate and o_shippriority - * After that, the reduce function calculates the revenue. - */ - joined = joined - .groupBy(0, 2, 3) - .reduce(new ReduceFunction() { - @Override - public ShippingPriorityItem reduce(ShippingPriorityItem value1, ShippingPriorityItem value2) { - value1.setRevenue(value1.getRevenue() + value2.getRevenue()); - return value1; - } - }); - - if (resultPath == null) { - joined.print(); } else { - joined.writeAsCsv(resultPath, "\n", "|"); + System.err.println("This program expects data from the TPC-H benchmark as input data.\n" + + " Due to legal restrictions, we can not ship generated data.\n" + + " You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + + " Usage: TPCHQuery3 "); + System.exit(1); } - - env.execute(); } + + private static DataSet getLineitemDataSet(ExecutionEnvironment env) { + return env.readCsvFile(lineitemPath) + .fieldDelimiter('|') + .includeFields("1000011000100000") + .tupleType(Lineitem.class); + } + + private static DataSet getCustomerDataSet(ExecutionEnvironment env) { + return env.readCsvFile(customerPath) + .fieldDelimiter('|') + .includeFields("10000010") + .tupleType(Customer.class); + } + + private static DataSet getOrdersDataSet(ExecutionEnvironment env) { + return env.readCsvFile(ordersPath) + .fieldDelimiter('|') + .includeFields("100010010") + .tupleType(Order.class); + } + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java index 6d17e45e4c817..d7eb4a95f3e36 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java @@ -13,6 +13,8 @@ package eu.stratosphere.example.java.relational; +import java.util.Iterator; + import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.ExecutionEnvironment; import eu.stratosphere.api.java.functions.CoGroupFunction; @@ -20,57 +22,135 @@ import eu.stratosphere.api.java.tuple.Tuple1; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.example.java.relational.util.WebLogData; +import eu.stratosphere.example.java.relational.util.WebLogDataGenerator; import eu.stratosphere.util.Collector; -import java.util.Iterator; - /** - * Implements the following relational OLAP query as Stratosphere program: + * This program processes web logs and relational data. + * It implements the following relational query: * *
- * SELECT r.pageURL, r.pageRank, r.avgDuration
- * FROM Documents d JOIN Rankings r
- * 	ON d.url = r.url
- * WHERE CONTAINS(d.text, [keywords])
- * 	AND r.rank > [rank]
- * 	AND NOT EXISTS (
- * 		SELECT * FROM Visits v
- * 		WHERE v.destUrl = d.url
- * 			AND v.visitDate < [date]);
- *  * 
+ * SELECT + * r.pageURL, + * r.pageRank, + * r.avgDuration + * FROM documents d JOIN rankings r + * ON d.url = r.url + * WHERE CONTAINS(d.text, [keywords]) + * AND r.rank > [rank] + * AND NOT EXISTS + * ( + * SELECT * FROM Visits v + * WHERE v.destUrl = d.url + * AND v.visitDate < [date] + * ); + * * - * Table Schemas:
+ * 

+ * Input files are plain text CSV files using the pipe character ('|') as field separator. + * The tables referenced in the query can be generated using the {@link WebLogDataGenerator} and + * have the following schemas + *

  * CREATE TABLE Documents (
- * 					url VARCHAR(100) PRIMARY KEY,
- * 					contents TEXT );
+ *                url VARCHAR(100) PRIMARY KEY,
+ *                contents TEXT );
  *
  * CREATE TABLE Rankings (
- * 					pageRank INT,
- * 					pageURL VARCHAR(100) PRIMARY KEY,
- * 					avgDuration INT );
+ *                pageRank INT,
+ *                pageURL VARCHAR(100) PRIMARY KEY,
+ *                avgDuration INT );
  *
  * CREATE TABLE Visits (
- * 					sourceIP VARCHAR(16),
- * 					destURL VARCHAR(100),
- * 					visitDate DATE,
- * 					adRevenue FLOAT,
- * 					userAgent VARCHAR(64),
- * 					countryCode VARCHAR(3),
- * 					languageCode VARCHAR(6),
- * 					searchWord VARCHAR(32),
- * 					duration INT );
+ *                sourceIP VARCHAR(16),
+ *                destURL VARCHAR(100),
+ *                visitDate DATE,
+ *                adRevenue FLOAT,
+ *                userAgent VARCHAR(64),
+ *                countryCode VARCHAR(3),
+ *                languageCode VARCHAR(6),
+ *                searchWord VARCHAR(32),
+ *                duration INT );
  * 
+ * + *

+ * This example shows how to use: + *

    + *
  • tuple data types + *
  • projection and join projection + *
  • the CoGroup transformation for an anti-join + *
+ * */ @SuppressWarnings("serial") public class WebLogAnalysis { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + parseParameters(args); + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + // get input data + DataSet> documents = getDocumentsDataSet(env); + DataSet> ranks = getRanksDataSet(env); + DataSet> visits = getVisitsDataSet(env); + + // Create DataSet for filtering the entries from the documents relation + DataSet> filterDocs = documents + .filter(new FilterDocs()) + .project(0).types(String.class); + + // Create DataSet for filtering the entries from the ranks relation + DataSet> filterRanks = ranks + .filter(new FilterRanks()); + + // Create DataSet for filtering the entries from the visits relation + DataSet> filterVisits = visits + .filter(new FilterVisits()) + .project(0).types(String.class); + + // Create DataSet to join the filtered documents and ranks relation + DataSet> joinDocsRanks = + filterDocs.join(filterRanks) + .where(0).equalTo(1) + .projectSecond(0,1,2) + .types(Integer.class, String.class, Integer.class); + + // Create DataSet to realize a anti join between the joined + // documents and ranks relation and the filtered visits relation + DataSet> result = + joinDocsRanks.coGroup(filterVisits) + .where(1).equalTo(0) + .with(new AntiJoinVisits()); + + // emit result + if(fileOutput) { + result.writeAsCsv(outputPath, "\n", "|"); + } else { + result.print(); + } + + // execute program + env.execute("WebLogAnalysis Example"); + + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + /** * MapFunction that filters for documents that contain a certain set of * keywords. */ public static class FilterDocs extends FilterFunction> { - private static final String[] KEYWORDS = { " editors ", " oscillations ", " convection " }; + private static final String[] KEYWORDS = { " editors ", " oscillations " }; /** * Filters for documents that contain all of the given keywords and projects the records on the URL field. @@ -98,7 +178,7 @@ public boolean filter(Tuple2 value) throws Exception { */ public static class FilterRanks extends FilterFunction> { - private static final int RANKFILTER = 50; + private static final int RANKFILTER = 40; /** * Filters for records of the rank relation where the rank is greater @@ -121,7 +201,7 @@ public boolean filter(Tuple3 value) throws Exception { */ public static class FilterVisits extends FilterFunction> { - private static final int YEARFILTER = 2010; + private static final int YEARFILTER = 2007; /** * Filters for records of the visits relation where the year of visit is equal to a @@ -169,76 +249,68 @@ public void coGroup(Iterator> ranks, Iterator ."); - System.err.println(" If is \"STDOUT\", prints result to the command line."); - return; - } + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static boolean fileOutput = false; + private static String documentsPath; + private static String ranksPath; + private static String visitsPath; + private static String outputPath; + + private static void parseParameters(String[] args) { - String docsInput = args[0]; - String ranksInput = args[1]; - String visitsInput = args[2]; - String output = args[3]; - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - /* - * Output Format: - * 0: URL - * 1: DOCUMENT_TEXT - */ - // Create DataSet for documents relation - DataSet> docs = env.readCsvFile(docsInput) - .fieldDelimiter('|') - .types(String.class, String.class); - - /* - * Output Format: - * 0: RANK - * 1: URL - * 2: AVG_DURATION - */ - // Create DataSet for ranks relation - DataSet> ranks = env.readCsvFile(ranksInput) - .fieldDelimiter('|') - .types(Integer.class, String.class, Integer.class); - - /* - * Output Format: - * 0: URL - * 1: DATE - */ - // Create DataSet for visits relation - DataSet> visits = env.readCsvFile(visitsInput) - .fieldDelimiter('|') - .includeFields("011000000") - .types(String.class, String.class); - - // Create DataSet for filtering the entries from the documents relation - DataSet> filterDocs = docs.filter(new FilterDocs()).project(0).types(String.class); - - // Create DataSet for filtering the entries from the ranks relation - DataSet> filterRanks = ranks.filter(new FilterRanks()); - - // Create DataSet for filtering the entries from the visits relation - DataSet> filterVisits = visits.filter(new FilterVisits()).project(0).types(String.class); - - // Create DataSet to join the filtered documents and ranks relation - DataSet> joinDocsRanks = filterDocs.join(filterRanks) - .where(0).equalTo(1).projectSecond(0,1,2).types(Integer.class, String.class, Integer.class); - - // Create DataSet to realize a anti join between the joined - // documents and ranks relation and the filtered visits relation - DataSet> antiJoinVisits = joinDocsRanks.coGroup(filterVisits) - .where(1).equalTo(0).with(new AntiJoinVisits()); - - if (output.equals("STDOUT")) { - antiJoinVisits.print(); + if(args.length > 0) { + fileOutput = true; + if(args.length == 4) { + documentsPath = args[0]; + ranksPath = args[1]; + visitsPath = args[2]; + outputPath = args[3]; + } else { + System.err.println("Usage: WebLogAnalysis "); + System.exit(1); + } } else { - antiJoinVisits.writeAsCsv(output, "\n", "|"); + System.out.println("Executing WebLog Analysis example with built-in default data."); + System.out.println(" Provide parameters to read input data from files."); + System.out.println(" Usage: WebLogAnalysis "); } + } + + private static DataSet> getDocumentsDataSet(ExecutionEnvironment env) { + // Create DataSet for documents relation (URL, Doc-Text) + if(fileOutput) { + return env.readCsvFile(documentsPath) + .fieldDelimiter('|') + .types(String.class, String.class); + } else { + return WebLogData.getDocumentDataSet(env); + } + } + + private static DataSet> getRanksDataSet(ExecutionEnvironment env) { + // Create DataSet for ranks relation (Rank, URL, Avg-Visit-Duration) + if(fileOutput) { + return env.readCsvFile(ranksPath) + .fieldDelimiter('|') + .types(Integer.class, String.class, Integer.class); + } else { + return WebLogData.getRankDataSet(env); + } + } - env.execute(); + private static DataSet> getVisitsDataSet(ExecutionEnvironment env) { + // Create DataSet for visits relation (URL, Date) + if(fileOutput) { + return env.readCsvFile(visitsPath) + .fieldDelimiter('|') + .includeFields("011000000") + .types(String.class, String.class); + } else { + return WebLogData.getVisitDataSet(env); + } } + } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogData.java new file mode 100644 index 0000000000000..00045967cd413 --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogData.java @@ -0,0 +1,401 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.relational.util; + +import java.util.ArrayList; +import java.util.List; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.api.java.tuple.Tuple3; + +public class WebLogData { + + public static DataSet> getDocumentDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(100); + data.add(new Tuple2("url_0","dolor ad amet enim laoreet nostrud veniam aliquip ex nonummy diam dolore tincidunt tation exerci exerci wisi dolor nostrud ")); + data.add(new Tuple2("url_1","wisi minim adipiscing nibh adipiscing ut nibh Lorem Ut nonummy euismod nibh wisi sit consectetuer exerci sed aliquip aliquip dolore aliquam enim dolore veniam aliquam euismod suscipit ad adipiscing exerci aliquip consectetuer euismod aliquip ad exerci ex nibh ex erat exerci laoreet lobortis quis ")); + data.add(new Tuple2("url_2","diam sed convection aliquip amet commodo nonummy sed sed commodo commodo diam commodo adipiscing ad exerci magna exerci tation quis lobortis ")); + data.add(new Tuple2("url_3","exerci suscipit sed lobortis amet lobortis aliquip nibh nostrud ad convection commodo ad nibh sed minim amet ad ea ea ")); + data.add(new Tuple2("url_4","sit enim dolor quis laoreet ullamcorper veniam adipiscing ex quis commodo ")); + data.add(new Tuple2("url_5","elit aliquip ea nisl oscillations sit dolor ipsum tincidunt ullamcorper dolore enim adipiscing laoreet elit ea volutpat adipiscing ea nibh nostrud Ut aliquam veniam Lorem laoreet veniam aliquip ")); + data.add(new Tuple2("url_6","consectetuer ad sed suscipit euismod aliquip quis ullamcorper oscillations tation consectetuer tation amet suscipit nibh enim nonummy veniam commodo commodo diam euismod dolor Ut aliquip diam ex ad nonummy ad tincidunt minim exerci consectetuer veniam convection aliquam ut ut Lorem euismod sed ipsum volutpat ")); + data.add(new Tuple2("url_7","Ut volutpat veniam ut consectetuer diam ut aliquam dolor nostrud erat consectetuer adipiscing exerci consectetuer Ut ullamcorper suscipit aliquam sed dolor nisl ")); + data.add(new Tuple2("url_8","suscipit amet wisi nisl veniam lobortis sit Lorem aliquam nostrud aliquam ipsum ut laoreet suscipit Lorem laoreet editors adipiscing ullamcorper veniam erat consectetuer ut lobortis dolore elit sed tincidunt ipsum tation ullamcorper nonummy adipiscing ex ad laoreet ipsum suscipit lobortis lobortis Ut nonummy adipiscing erat volutpat aliquam ")); + data.add(new Tuple2("url_9","nonummy commodo tation editors ut quis sit quis lobortis ea dolore oscillations diam ad dolor lobortis nisl ad veniam ullamcorper quis magna volutpat sit ipsum consectetuer dolore exerci commodo magna erat enim ut suscipit ")); + data.add(new Tuple2("url_10","amet erat magna consectetuer tation tation aliquip nibh aliquam sed adipiscing ut commodo ex erat tincidunt aliquam ipsum Ut Ut sit tincidunt adipiscing suscipit minim sed erat dolor consectetuer Lorem consectetuer Lorem amet nibh diam ea ex enim suscipit wisi dolor nonummy magna enim euismod ullamcorper ut suscipit adipiscing ")); + data.add(new Tuple2("url_11","ex quis exerci tation diam elit nostrud nostrud ut ipsum elit amet diam laoreet amet consectetuer volutpat sed lobortis ")); + data.add(new Tuple2("url_12","elit suscipit sit ullamcorper ut ad erat ut dolor nostrud quis nisl enim erat dolor convection ad minim ut veniam nostrud sed editors adipiscing volutpat Ut aliquip commodo sed euismod adipiscing erat adipiscing dolore nostrud minim sed lobortis ea diam ")); + data.add(new Tuple2("url_13","enim ut quis commodo veniam minim erat lobortis ad diam ex dolor tincidunt exerci ut aliquip tincidunt minim ut magna sed enim wisi veniam oscillations Lorem consectetuer ")); + data.add(new Tuple2("url_14","nibh ipsum ullamcorper volutpat ut wisi dolor quis amet euismod quis ipsum ipsum minim tation volutpat sit exerci volutpat amet nonummy euismod veniam consectetuer sit consectetuer tincidunt nibh aliquam lobortis tation veniam ut ullamcorper wisi magna Ut volutpat consectetuer erat quis dolore ea tation ")); + data.add(new Tuple2("url_15","ad wisi sed enim aliquam oscillations nibh Lorem lobortis veniam nibh laoreet nonummy sed nibh Lorem adipiscing diam magna nostrud magna oscillations ut oscillations elit nostrud diam editors Lorem ")); + data.add(new Tuple2("url_16","nostrud volutpat veniam exerci tincidunt nostrud quis elit ipsum ea nonummy volutpat dolor elit lobortis magna nisl ut ullamcorper magna Lorem exerci nibh nisl magna editors erat aliquam aliquam ullamcorper sit aliquam sit nostrud oscillations consectetuer adipiscing suscipit convection exerci ea ullamcorper ex nisl ")); + data.add(new Tuple2("url_17","ad ex aliquam erat aliquam elit veniam laoreet ut amet amet nostrud ut adipiscing Ut Lorem suscipit ex magna ullamcorper aliquam ullamcorper ullamcorper amet amet commodo aliquam volutpat nonummy nonummy tincidunt amet tation tincidunt volutpat ut veniam nisl erat dolor enim nonummy nostrud adipiscing laoreet adipiscing ")); + data.add(new Tuple2("url_18","lobortis ipsum ex tincidunt tincidunt editors euismod consectetuer ipsum adipiscing lobortis exerci adipiscing nonummy nisl dolore nonummy erat exerci nisl ut dolore wisi volutpat lobortis magna ")); + data.add(new Tuple2("url_19","ipsum tation laoreet tation adipiscing wisi nibh diam Ut suscipit ad wisi ")); + data.add(new Tuple2("url_20","diam Lorem enim wisi ad lobortis dolor Ut ipsum amet dolore consectetuer nisl exerci nisl nonummy minim Ut erat oscillations ut Lorem nostrud dolore Ut dolore exerci ad ipsum dolore ex dolore aliquip sed aliquam ex aliquip magna amet ex dolore oscillations aliquip tation magna Ut ")); + data.add(new Tuple2("url_21","lobortis ut amet ex nisl ullamcorper tincidunt ut elit diam quis suscipit ad amet ipsum magna Ut ex tincidunt ")); + data.add(new Tuple2("url_22","amet commodo nisl ad quis lobortis ut commodo sit ut erat exerci lobortis suscipit nibh ut nostrud ut adipiscing commodo commodo quis quis nostrud nisl ipsum nostrud laoreet Lorem nostrud erat nostrud amet consectetuer laoreet oscillations wisi sit magna nibh amet ")); + data.add(new Tuple2("url_23","adipiscing suscipit suscipit aliquip suscipit consectetuer minim magna ea erat nibh sit suscipit sed dolor oscillations nonummy volutpat ut tincidunt ")); + data.add(new Tuple2("url_24","commodo sed tincidunt aliquip aliquip dolore commodo nonummy sed erat ut ex exerci dolore adipiscing tincidunt ex diam amet aliquam ")); + data.add(new Tuple2("url_25","consectetuer consectetuer exerci quis ea veniam aliquam laoreet minim ex ")); + data.add(new Tuple2("url_26","dolor exerci euismod minim magna quis erat consectetuer sed ex erat dolore quis ut oscillations ullamcorper Lorem exerci ex nibh ut exerci ullamcorper veniam nibh ut commodo ut Ut nostrud tincidunt tincidunt ad dolore Lorem ea tation enim erat nibh ut ea nonummy sed sed wisi nisl dolore ")); + data.add(new Tuple2("url_27","amet elit ea ea nostrud editors Ut nostrud amet laoreet adipiscing ut nisl nonummy tincidunt ea ipsum ex dolore dolore oscillations sit minim Ut wisi ut laoreet minim elit ")); + data.add(new Tuple2("url_28","wisi exerci volutpat Ut nostrud euismod minim Ut sit euismod ut ea magna consectetuer nisl ad minim tation nisl adipiscing Lorem aliquam quis exerci erat minim aliquip sit Lorem wisi wisi ut ")); + data.add(new Tuple2("url_29","amet sed laoreet amet aliquam minim enim tincidunt Lorem sit aliquip amet suscipit ut laoreet elit suscipit erat ut tincidunt suscipit ipsum sed euismod elit dolore euismod dolore ut dolor nostrud ipsum tincidunt commodo adipiscing aliquam ut wisi dolor dolor suscipit ")); + data.add(new Tuple2("url_30","euismod Lorem ex tincidunt amet enim minim suscipit exerci diam veniam amet nostrud ea ea ")); + data.add(new Tuple2("url_31","ex ipsum sit euismod euismod ullamcorper tincidunt ut wisi ea adipiscing sed diam tation ipsum dolor aliquam veniam nonummy aliquip aliquip Lorem ut minim nisl tation sit exerci ullamcorper Ut dolor euismod aliquam consectetuer ad nonummy commodo exerci ")); + data.add(new Tuple2("url_32","volutpat ipsum lobortis nisl veniam minim adipiscing dolor editors quis nostrud amet nostrud ")); + data.add(new Tuple2("url_33","commodo wisi aliquip ut aliquam sed nostrud ex diam ad nostrud enim ut amet enim ea ad sed tation nostrud suscipit ea magna magna Lorem amet lobortis ut quis nibh aliquam aliquam exerci aliquip lobortis consectetuer enim wisi ea nisl laoreet erat dolore ")); + data.add(new Tuple2("url_34","tincidunt adipiscing enim tation nibh Ut dolore tincidunt tation laoreet suscipit minim aliquam volutpat laoreet suscipit tincidunt nibh ut ut sit nostrud nonummy tincidunt exerci sit ad sed consectetuer minim dolor dolore laoreet nostrud nibh laoreet ea adipiscing exerci dolore ipsum ")); + data.add(new Tuple2("url_35","tation ut erat ut tation dolor Lorem laoreet Lorem elit adipiscing wisi aliquip nostrud elit Ut volutpat ea aliquam aliquip ")); + data.add(new Tuple2("url_36","lobortis enim ullamcorper adipiscing consectetuer aliquip wisi enim minim Ut minim elit elit aliquam exerci ullamcorper amet lobortis adipiscing diam laoreet consectetuer nostrud diam diam amet ut enim ullamcorper aliquip diam ut nostrud diam magna amet nonummy commodo wisi enim ullamcorper suscipit euismod dolore tincidunt magna suscipit elit ")); + data.add(new Tuple2("url_37","elit adipiscing nisl nisl ex aliquip nibh sed ut ad Lorem elit consectetuer ad volutpat lobortis amet veniam ipsum nibh ut consectetuer editors ad aliquam ")); + data.add(new Tuple2("url_38","elit quis nibh adipiscing sit consectetuer ut euismod quis tincidunt quis nisl consectetuer dolor diam suscipit quis dolore Lorem suscipit nonummy sed ex ")); + data.add(new Tuple2("url_39","nisl sit consectetuer elit oscillations enim ipsum enim nostrud adipiscing nostrud editors aliquam ")); + data.add(new Tuple2("url_40","sed wisi dolor diam commodo ullamcorper commodo nostrud ullamcorper laoreet minim dolore suscipit laoreet tation aliquip ")); + data.add(new Tuple2("url_41","ad consectetuer exerci nisl exerci amet enim diam lobortis Lorem ex volutpat volutpat nibh aliquam ut ullamcorper volutpat nostrud ut adipiscing ullamcorper ")); + data.add(new Tuple2("url_42","minim laoreet tation magna veniam ut ea sit ipsum tincidunt Ut amet ex aliquip ex euismod exerci wisi elit editors ad amet veniam ad editors ")); + data.add(new Tuple2("url_43","ut nisl ad ullamcorper nibh Ut editors exerci enim exerci ea laoreet veniam ea amet exerci volutpat amet ad ")); + data.add(new Tuple2("url_44","volutpat tincidunt enim amet sed tincidunt consectetuer ullamcorper nisl Ut adipiscing tation ad ad amet nonummy elit erat nibh Lorem erat elit laoreet consectetuer sed aliquip nostrud ")); + data.add(new Tuple2("url_45","sed aliquam ut ut consectetuer wisi euismod enim erat euismod quis exerci amet tation sit ")); + data.add(new Tuple2("url_46","lobortis oscillations tation aliquam dolore Lorem aliquip tation exerci ullamcorper aliquam aliquip lobortis ex tation dolor ut ut sed suscipit nisl ullamcorper sed editors laoreet aliquip enim dolor veniam tincidunt sed euismod tation ")); + data.add(new Tuple2("url_47","Lorem Lorem ut wisi ad ut tation consectetuer exerci convection tation ullamcorper sed dolore quis aliquam ipsum lobortis commodo nonummy ")); + data.add(new Tuple2("url_48","laoreet minim veniam nisl elit sit amet commodo ex ullamcorper suscipit aliquip laoreet convection Ut ex minim aliquam ")); + data.add(new Tuple2("url_49","lobortis nonummy minim amet sit veniam quis consectetuer tincidunt laoreet quis ")); + data.add(new Tuple2("url_50","lobortis nisl commodo dolor amet nibh editors enim magna minim elit euismod diam laoreet laoreet ad minim sed ut Ut lobortis adipiscing quis sed ut aliquam oscillations exerci tation consectetuer lobortis elit tincidunt consectetuer minim amet dolore quis aliquam Ut exerci sed aliquam quis quis ullamcorper Ut ex tincidunt ")); + data.add(new Tuple2("url_51","nostrud nisl ea erat ut suscipit Ut sit oscillations ullamcorper nonummy magna lobortis dolore editors tincidunt nostrud suscipit ex quis tation ut sit amet nostrud laoreet ex tincidunt ")); + data.add(new Tuple2("url_52","ea tation commodo elit sed ex sed quis enim nisl magna laoreet adipiscing amet sit nostrud consectetuer nibh tincidunt veniam ex veniam euismod exerci sed dolore suscipit nisl tincidunt euismod quis Ut enim euismod dolor diam exerci magna exerci ut exerci nisl ")); + data.add(new Tuple2("url_53","volutpat amet Ut lobortis dolor tation minim nonummy lobortis convection nostrud ")); + data.add(new Tuple2("url_54","ullamcorper commodo Ut amet sit nostrud aliquam ad amet wisi enim nostrud ipsum nisl veniam erat aliquam ex aliquam dolor dolor ut consectetuer euismod exerci elit exerci Ut ea minim enim consectetuer ad consectetuer nonummy convection adipiscing ad ullamcorper lobortis nonummy laoreet nonummy aliquam ullamcorper ad nostrud amet ")); + data.add(new Tuple2("url_55","wisi magna editors amet aliquam diam amet aliquip nisl consectetuer laoreet nonummy suscipit euismod diam enim tation elit ut lobortis quis euismod suscipit nostrud ea ea commodo lobortis dolore Ut nisl nostrud dolor laoreet euismod ea dolore aliquam ut Lorem exerci ex sit ")); + data.add(new Tuple2("url_56","ex dolor veniam wisi laoreet ut exerci diam ad ex ut ut laoreet ut nisl ullamcorper nisl ")); + data.add(new Tuple2("url_57","diam adipiscing Ut ut Lorem amet erat elit erat magna adipiscing euismod elit ullamcorper nostrud aliquam dolor ullamcorper sit tation tation ")); + data.add(new Tuple2("url_58","laoreet convection veniam lobortis dolore ut nonummy commodo erat lobortis veniam nostrud dolore minim commodo ut consectetuer magna erat ea dolore Lorem suscipit ex ipsum exerci sed enim ea tation suscipit enim adipiscing ")); + data.add(new Tuple2("url_59","amet ut ut Ut ad dolor quis ad magna exerci suscipit magna nibh commodo euismod amet euismod wisi diam suscipit dolore Lorem dolor ex amet exerci aliquip ut ut lobortis quis elit minim sed Lorem ")); + data.add(new Tuple2("url_60","ut ut amet ullamcorper amet euismod dolor amet elit exerci adipiscing sed suscipit sed exerci wisi diam veniam wisi suscipit ut quis nibh ullamcorper ex quis magna dolore volutpat editors minim ut sit aliquip oscillations nisl ipsum ")); + data.add(new Tuple2("url_61","nibh nostrud tincidunt lobortis adipiscing adipiscing ullamcorper ullamcorper ipsum nisl ullamcorper aliquip laoreet commodo ut tation wisi diam commodo aliquip commodo suscipit tincidunt volutpat elit enim laoreet ut nostrud ad nonummy ipsum ")); + data.add(new Tuple2("url_62","Ut ut minim enim amet euismod erat elit commodo consectetuer Ut quis dolor ex diam quis wisi tation tincidunt laoreet volutpat ")); + data.add(new Tuple2("url_63","ut erat volutpat euismod amet ea nonummy lobortis ut Ut ea veniam sed veniam nostrud ")); + data.add(new Tuple2("url_64","tation dolor suscipit minim nisl wisi consectetuer aliquip tation Ut commodo ut dolore consectetuer elit wisi nisl ipsum ")); + data.add(new Tuple2("url_65","ullamcorper nisl Lorem magna tation veniam aliquam diam amet euismod ")); + data.add(new Tuple2("url_66","euismod aliquam tincidunt Ut volutpat ea lobortis sit ut volutpat ut lobortis ut lobortis ut nisl amet dolor sed ipsum enim ullamcorper diam euismod nostrud wisi erat quis diam nibh Ut dolore sed amet tation enim diam ")); + data.add(new Tuple2("url_67","amet minim minim amet laoreet Lorem aliquam veniam elit volutpat magna adipiscing enim enim euismod laoreet sed ex sed aliquam ad ea ut adipiscing suscipit ex minim dolore minim ea laoreet nisl ")); + data.add(new Tuple2("url_68","aliquam ea volutpat ut wisi tation tation nibh nisl erat laoreet ea volutpat dolor dolor aliquam exerci quis ullamcorper aliquam ut quis suscipit ")); + data.add(new Tuple2("url_69","quis exerci ut aliquip wisi dolore magna nibh consectetuer magna tation ullamcorper lobortis sed amet adipiscing minim suscipit nibh nibh nostrud euismod enim ")); + data.add(new Tuple2("url_70","tation enim consectetuer adipiscing wisi laoreet diam aliquip nostrud elit nostrud aliquip ea minim amet diam dolore ")); + data.add(new Tuple2("url_71","consectetuer tincidunt nibh amet tation nonummy sit tation diam sed diam tation ")); + data.add(new Tuple2("url_72","Lorem ut nostrud nonummy minim quis euismod lobortis nostrud nonummy adipiscing tincidunt consectetuer ut nibh ad suscipit dolor ut elit dolore amet ut quis tation ullamcorper nonummy laoreet ullamcorper aliquam dolore convection dolor tincidunt ut ullamcorper ex dolor suscipit erat oscillations ad ")); + data.add(new Tuple2("url_73","elit Ut commodo ut ullamcorper ullamcorper ut euismod commodo diam aliquip suscipit consectetuer exerci tation nostrud ut wisi exerci sed ut elit sed volutpat Lorem nibh laoreet consectetuer ex Lorem elit aliquam commodo lobortis ad ")); + data.add(new Tuple2("url_74","quis magna laoreet commodo aliquam nisl ullamcorper veniam tation wisi consectetuer commodo consectetuer ad dolore aliquam dolor elit amet sit amet nibh commodo erat veniam aliquip dolore ad magna ad ipsum Ut exerci ea volutpat nisl amet nostrud sit ")); + data.add(new Tuple2("url_75","tincidunt suscipit sit aliquip aliquam adipiscing dolore exerci Ut suscipit ut sit laoreet suscipit wisi sit enim nonummy consectetuer dolore editors ")); + data.add(new Tuple2("url_76","veniam ullamcorper tation sit suscipit dolor suscipit veniam sit Lorem quis sed nostrud ad tincidunt elit adipiscing ")); + data.add(new Tuple2("url_77","volutpat sit amet veniam quis ipsum nibh elit enim commodo magna veniam magna convection ")); + data.add(new Tuple2("url_78","tation dolore minim elit nisl volutpat tation laoreet enim nostrud exerci dolore tincidunt aliquip Lorem ipsum nostrud quis adipiscing ullamcorper erat lobortis tation commodo Ut ipsum commodo magna ad ipsum ut enim ")); + data.add(new Tuple2("url_79","lobortis amet elit Lorem amet nonummy commodo tation ex ea amet Lorem ea nonummy commodo veniam volutpat nibh wisi ad ipsum euismod ea convection nostrud nisl erat veniam Ut aliquip ad aliquip editors wisi magna tation nostrud nonummy adipiscing ullamcorper aliquip ")); + data.add(new Tuple2("url_80","tincidunt nostrud nostrud magna ea euismod ea consectetuer nisl exerci ea dolor nisl commodo ex erat ipsum exerci suscipit ad nisl ea nonummy suscipit adipiscing laoreet sit euismod nibh adipiscing sed minim commodo amet ")); + data.add(new Tuple2("url_81","nostrud erat ut sed editors erat amet magna lobortis diam laoreet dolor amet nibh ut ipsum ipsum amet ut sed ut exerci elit suscipit wisi magna ut veniam nisl commodo enim adipiscing laoreet ad Lorem oscillations ")); + data.add(new Tuple2("url_82","quis commodo nibh nibh volutpat suscipit dolore magna tincidunt nibh ut ad ullamcorper ullamcorper quis enim ad ut tation minim laoreet veniam dolor sed tincidunt exerci exerci nostrud ullamcorper amet ut ut ullamcorper ")); + data.add(new Tuple2("url_83","sit suscipit volutpat elit tation elit sed sed dolor ex ex ipsum euismod laoreet magna lobortis ad ")); + data.add(new Tuple2("url_84","lobortis ipsum euismod enim ea tation veniam tation oscillations aliquip consectetuer euismod ut sed lobortis tation oscillations commodo euismod laoreet suscipit amet elit ullamcorper volutpat aliquam ea enim ullamcorper consectetuer laoreet tation quis ut commodo erat euismod dolor laoreet ullamcorper laoreet ")); + data.add(new Tuple2("url_85","adipiscing sit quis commodo consectetuer quis enim euismod exerci nonummy ea nostrud Ut veniam sit aliquip nisl enim ")); + data.add(new Tuple2("url_86","nostrud dolore veniam veniam wisi aliquip adipiscing diam sed quis ullamcorper ")); + data.add(new Tuple2("url_87","quis Lorem suscipit Ut nibh diam euismod consectetuer lobortis ipsum sed suscipit consectetuer euismod laoreet ut wisi nisl elit quis commodo adipiscing adipiscing suscipit aliquam nisl quis magna ipsum enim ad quis ea magna Lorem nibh ea ")); + data.add(new Tuple2("url_88","euismod commodo sed tincidunt Ut veniam consectetuer quis erat ex ea erat laoreet commodo nibh minim ")); + data.add(new Tuple2("url_89","tation diam editors Ut enim nibh Lorem volutpat quis diam suscipit exerci wisi ad ")); + data.add(new Tuple2("url_90","volutpat editors ea nibh wisi ad amet volutpat nisl ullamcorper nibh volutpat minim ex ut sit veniam Lorem consectetuer quis ad sit suscipit volutpat wisi diam sed tincidunt ipsum minim convection ea diam oscillations quis lobortis ")); + data.add(new Tuple2("url_91","enim minim nonummy ea minim euismod adipiscing editors volutpat magna sit magna ut ipsum ut ")); + data.add(new Tuple2("url_92","nisl Ut commodo amet euismod lobortis ea ea wisi commodo Lorem sit ipsum volutpat nonummy exerci erat elit exerci magna ad erat enim laoreet quis nostrud wisi ut veniam amet ullamcorper lobortis ad suscipit volutpat veniam nostrud nibh quis ipsum dolore consectetuer veniam ipsum aliquip dolore sed laoreet ipsum ")); + data.add(new Tuple2("url_93","nonummy aliquam ad lobortis Lorem erat ad tation Lorem exerci ex ")); + data.add(new Tuple2("url_94","nonummy dolore commodo exerci ex quis ut suscipit elit laoreet sit tation magna veniam ea sit nonummy veniam Lorem quis nibh aliquip exerci amet ullamcorper adipiscing erat nisl editors diam commodo ad euismod adipiscing ea suscipit exerci aliquip volutpat tation enim volutpat sit ")); + data.add(new Tuple2("url_95","sit suscipit oscillations ipsum nibh dolor ea dolore ea elit ipsum minim editors magna consectetuer ullamcorper commodo nonummy sit nostrud aliquip sit erat ullamcorper ullamcorper nibh veniam erat quis dolore nonummy ")); + data.add(new Tuple2("url_96","nostrud quis ut volutpat magna ad quis adipiscing Lorem commodo exerci laoreet magna adipiscing erat quis wisi ea ea laoreet enim convection ad dolor nisl amet nibh aliquam adipiscing tincidunt minim diam Lorem commodo adipiscing volutpat ")); + data.add(new Tuple2("url_97","laoreet laoreet suscipit nostrud dolore adipiscing volutpat Ut sed nisl diam ullamcorper ex ut ut dolor amet nostrud euismod dolore veniam veniam enim tation veniam ea minim minim volutpat tincidunt ")); + data.add(new Tuple2("url_98","quis lobortis amet wisi nostrud ipsum aliquam convection tincidunt dolore ullamcorper nibh lobortis volutpat ea nostrud oscillations minim nonummy enim ad lobortis exerci ipsum ullamcorper nibh nonummy diam amet enim veniam ut nostrud ")); + data.add(new Tuple2("url_99","aliquam wisi suscipit commodo diam amet amet magna nisl enim nostrud tation nisl nostrud nibh ut ")); + + return env.fromCollection(data); + } + + public static DataSet> getRankDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(100); + data.add(new Tuple3(30,"url_0",43)); + data.add(new Tuple3(82,"url_1",39)); + data.add(new Tuple3(56,"url_2",31)); + data.add(new Tuple3(96,"url_3",36)); + data.add(new Tuple3(31,"url_4",36)); + data.add(new Tuple3(29,"url_5",6)); + data.add(new Tuple3(33,"url_6",48)); + data.add(new Tuple3(66,"url_7",40)); + data.add(new Tuple3(28,"url_8",51)); + data.add(new Tuple3(9,"url_9",4)); + data.add(new Tuple3(49,"url_10",24)); + data.add(new Tuple3(26,"url_11",12)); + data.add(new Tuple3(39,"url_12",46)); + data.add(new Tuple3(84,"url_13",53)); + data.add(new Tuple3(29,"url_14",50)); + data.add(new Tuple3(21,"url_15",12)); + data.add(new Tuple3(69,"url_16",34)); + data.add(new Tuple3(11,"url_17",38)); + data.add(new Tuple3(96,"url_18",13)); + data.add(new Tuple3(56,"url_19",48)); + data.add(new Tuple3(18,"url_20",36)); + data.add(new Tuple3(31,"url_21",21)); + data.add(new Tuple3(29,"url_22",11)); + data.add(new Tuple3(71,"url_23",30)); + data.add(new Tuple3(85,"url_24",48)); + data.add(new Tuple3(19,"url_25",45)); + data.add(new Tuple3(69,"url_26",9)); + data.add(new Tuple3(20,"url_27",51)); + data.add(new Tuple3(33,"url_28",46)); + data.add(new Tuple3(75,"url_29",38)); + data.add(new Tuple3(96,"url_30",51)); + data.add(new Tuple3(73,"url_31",40)); + data.add(new Tuple3(67,"url_32",16)); + data.add(new Tuple3(24,"url_33",24)); + data.add(new Tuple3(27,"url_34",35)); + data.add(new Tuple3(33,"url_35",35)); + data.add(new Tuple3(7,"url_36",22)); + data.add(new Tuple3(83,"url_37",41)); + data.add(new Tuple3(23,"url_38",49)); + data.add(new Tuple3(41,"url_39",33)); + data.add(new Tuple3(66,"url_40",38)); + data.add(new Tuple3(4,"url_41",52)); + data.add(new Tuple3(34,"url_42",4)); + data.add(new Tuple3(28,"url_43",12)); + data.add(new Tuple3(14,"url_44",14)); + data.add(new Tuple3(41,"url_45",11)); + data.add(new Tuple3(48,"url_46",37)); + data.add(new Tuple3(75,"url_47",41)); + data.add(new Tuple3(78,"url_48",3)); + data.add(new Tuple3(63,"url_49",28)); + + return env.fromCollection(data); + } + + public static DataSet> getVisitDataSet(ExecutionEnvironment env) { + + List> data = new ArrayList>(100); + data.add(new Tuple2("url_2","2003-12-17")); + data.add(new Tuple2("url_9","2008-11-11")); + data.add(new Tuple2("url_14","2003-11-5")); + data.add(new Tuple2("url_46","2009-2-16")); + data.add(new Tuple2("url_14","2004-11-9")); + data.add(new Tuple2("url_36","2001-3-9")); + data.add(new Tuple2("url_35","2006-8-13")); + data.add(new Tuple2("url_22","2008-1-18")); + data.add(new Tuple2("url_36","2002-3-9")); + data.add(new Tuple2("url_13","2007-7-17")); + data.add(new Tuple2("url_23","2009-6-16")); + data.add(new Tuple2("url_16","2000-7-15")); + data.add(new Tuple2("url_41","2002-5-10")); + data.add(new Tuple2("url_6","2004-11-9")); + data.add(new Tuple2("url_5","2003-6-7")); + data.add(new Tuple2("url_22","2002-11-5")); + data.add(new Tuple2("url_11","2007-7-21")); + data.add(new Tuple2("url_38","2009-12-2")); + data.add(new Tuple2("url_6","2004-11-2")); + data.add(new Tuple2("url_46","2000-6-4")); + data.add(new Tuple2("url_34","2003-9-2")); + data.add(new Tuple2("url_31","2008-2-24")); + data.add(new Tuple2("url_0","2003-2-2")); + data.add(new Tuple2("url_47","2003-7-8")); + data.add(new Tuple2("url_49","2009-9-13")); + data.add(new Tuple2("url_11","2003-4-2")); + data.add(new Tuple2("url_20","2000-6-18")); + data.add(new Tuple2("url_38","2000-2-22")); + data.add(new Tuple2("url_44","2009-2-17")); + data.add(new Tuple2("url_26","2000-6-21")); + data.add(new Tuple2("url_13","2000-11-25")); + data.add(new Tuple2("url_47","2005-4-19")); + data.add(new Tuple2("url_46","2008-1-7")); + data.add(new Tuple2("url_33","2004-12-24")); + data.add(new Tuple2("url_32","2009-2-8")); + data.add(new Tuple2("url_26","2000-9-21")); + data.add(new Tuple2("url_9","2002-8-18")); + data.add(new Tuple2("url_38","2002-11-27")); + data.add(new Tuple2("url_37","2008-2-26")); + data.add(new Tuple2("url_1","2007-3-22")); + data.add(new Tuple2("url_37","2002-3-20")); + data.add(new Tuple2("url_27","2008-11-12")); + data.add(new Tuple2("url_30","2000-12-16")); + data.add(new Tuple2("url_48","2000-12-17")); + data.add(new Tuple2("url_46","2008-4-16")); + data.add(new Tuple2("url_29","2006-3-9")); + data.add(new Tuple2("url_0","2007-7-26")); + data.add(new Tuple2("url_46","2009-12-15")); + data.add(new Tuple2("url_34","2002-2-13")); + data.add(new Tuple2("url_24","2009-3-1")); + data.add(new Tuple2("url_43","2007-11-4")); + data.add(new Tuple2("url_3","2004-2-16")); + data.add(new Tuple2("url_26","2000-10-26")); + data.add(new Tuple2("url_42","2004-7-14")); + data.add(new Tuple2("url_13","2004-9-10")); + data.add(new Tuple2("url_21","2000-2-21")); + data.add(new Tuple2("url_9","2006-6-5")); + data.add(new Tuple2("url_46","2001-12-17")); + data.add(new Tuple2("url_24","2006-12-8")); + data.add(new Tuple2("url_25","2006-9-2")); + data.add(new Tuple2("url_37","2002-6-26")); + data.add(new Tuple2("url_18","2006-6-2")); + data.add(new Tuple2("url_46","2003-5-24")); + data.add(new Tuple2("url_32","2000-10-17")); + data.add(new Tuple2("url_45","2002-1-12")); + data.add(new Tuple2("url_12","2005-12-13")); + data.add(new Tuple2("url_49","2009-3-9")); + data.add(new Tuple2("url_31","2001-9-19")); + data.add(new Tuple2("url_22","2002-7-9")); + data.add(new Tuple2("url_27","2005-2-3")); + data.add(new Tuple2("url_43","2008-7-15")); + data.add(new Tuple2("url_20","2000-3-23")); + data.add(new Tuple2("url_25","2002-5-8")); + data.add(new Tuple2("url_41","2004-4-27")); + data.add(new Tuple2("url_17","2008-7-17")); + data.add(new Tuple2("url_26","2009-12-16")); + data.add(new Tuple2("url_34","2006-2-10")); + data.add(new Tuple2("url_8","2009-4-14")); + data.add(new Tuple2("url_16","2000-2-24")); + data.add(new Tuple2("url_2","2009-2-10")); + data.add(new Tuple2("url_35","2003-2-24")); + data.add(new Tuple2("url_34","2008-3-16")); + data.add(new Tuple2("url_27","2005-1-5")); + data.add(new Tuple2("url_8","2008-12-10")); + data.add(new Tuple2("url_38","2009-2-11")); + data.add(new Tuple2("url_38","2006-11-3")); + data.add(new Tuple2("url_47","2003-2-13")); + data.add(new Tuple2("url_8","2008-11-17")); + data.add(new Tuple2("url_26","2009-5-11")); + data.add(new Tuple2("url_12","2007-11-26")); + data.add(new Tuple2("url_10","2003-1-13")); + data.add(new Tuple2("url_8","2005-9-23")); + data.add(new Tuple2("url_42","2001-4-5")); + data.add(new Tuple2("url_30","2009-12-10")); + data.add(new Tuple2("url_2","2003-1-3")); + data.add(new Tuple2("url_2","2009-2-19")); + data.add(new Tuple2("url_7","2000-6-25")); + data.add(new Tuple2("url_15","2004-9-26")); + data.add(new Tuple2("url_25","2009-10-5")); + data.add(new Tuple2("url_23","2009-8-9")); + data.add(new Tuple2("url_27","2004-4-3")); + data.add(new Tuple2("url_37","2008-6-9")); + data.add(new Tuple2("url_9","2002-5-25")); + data.add(new Tuple2("url_43","2009-5-18")); + data.add(new Tuple2("url_21","2008-4-19")); + data.add(new Tuple2("url_12","2001-12-25")); + data.add(new Tuple2("url_16","2006-9-25")); + data.add(new Tuple2("url_27","2002-1-2")); + data.add(new Tuple2("url_2","2009-1-21")); + data.add(new Tuple2("url_31","2009-3-20")); + data.add(new Tuple2("url_42","2002-3-1")); + data.add(new Tuple2("url_31","2001-11-26")); + data.add(new Tuple2("url_20","2003-5-15")); + data.add(new Tuple2("url_32","2004-1-22")); + data.add(new Tuple2("url_28","2008-9-16")); + data.add(new Tuple2("url_27","2006-7-3")); + data.add(new Tuple2("url_11","2008-12-26")); + data.add(new Tuple2("url_15","2004-8-16")); + data.add(new Tuple2("url_34","2002-10-5")); + data.add(new Tuple2("url_44","2000-2-15")); + data.add(new Tuple2("url_9","2000-10-23")); + data.add(new Tuple2("url_45","2005-4-24")); + data.add(new Tuple2("url_0","2006-8-7")); + data.add(new Tuple2("url_48","2003-8-7")); + data.add(new Tuple2("url_8","2007-12-13")); + data.add(new Tuple2("url_42","2003-8-2")); + data.add(new Tuple2("url_25","2008-3-5")); + data.add(new Tuple2("url_3","2007-3-9")); + data.add(new Tuple2("url_49","2003-10-7")); + data.add(new Tuple2("url_18","2007-12-6")); + data.add(new Tuple2("url_3","2006-7-5")); + data.add(new Tuple2("url_27","2000-9-14")); + data.add(new Tuple2("url_42","2002-10-20")); + data.add(new Tuple2("url_44","2007-1-13")); + data.add(new Tuple2("url_6","2003-1-21")); + data.add(new Tuple2("url_40","2009-10-20")); + data.add(new Tuple2("url_28","2009-6-17")); + data.add(new Tuple2("url_22","2000-2-17")); + data.add(new Tuple2("url_3","2005-1-15")); + data.add(new Tuple2("url_9","2008-12-9")); + data.add(new Tuple2("url_9","2005-2-19")); + data.add(new Tuple2("url_28","2000-4-22")); + data.add(new Tuple2("url_44","2001-9-9")); + data.add(new Tuple2("url_43","2008-6-21")); + data.add(new Tuple2("url_39","2008-5-9")); + data.add(new Tuple2("url_15","2006-9-15")); + data.add(new Tuple2("url_23","2001-12-18")); + data.add(new Tuple2("url_14","2002-5-23")); + data.add(new Tuple2("url_11","2007-7-11")); + data.add(new Tuple2("url_34","2000-12-8")); + data.add(new Tuple2("url_47","2005-7-3")); + data.add(new Tuple2("url_38","2004-3-26")); + data.add(new Tuple2("url_19","2003-9-14")); + data.add(new Tuple2("url_24","2007-7-16")); + data.add(new Tuple2("url_40","2008-8-21")); + data.add(new Tuple2("url_17","2007-12-4")); + data.add(new Tuple2("url_25","2006-6-24")); + data.add(new Tuple2("url_2","2000-10-8")); + data.add(new Tuple2("url_12","2008-6-10")); + data.add(new Tuple2("url_11","2004-11-24")); + data.add(new Tuple2("url_13","2005-11-3")); + data.add(new Tuple2("url_43","2005-1-2")); + data.add(new Tuple2("url_14","2008-6-12")); + data.add(new Tuple2("url_43","2001-8-27")); + data.add(new Tuple2("url_45","2000-3-3")); + data.add(new Tuple2("url_0","2006-9-27")); + data.add(new Tuple2("url_22","2007-12-18")); + data.add(new Tuple2("url_25","2006-4-4")); + data.add(new Tuple2("url_32","2001-6-25")); + data.add(new Tuple2("url_6","2007-6-9")); + data.add(new Tuple2("url_8","2009-10-3")); + data.add(new Tuple2("url_15","2003-2-23")); + data.add(new Tuple2("url_37","2000-5-6")); + data.add(new Tuple2("url_27","2004-3-21")); + data.add(new Tuple2("url_17","2005-6-20")); + data.add(new Tuple2("url_2","2004-2-27")); + data.add(new Tuple2("url_36","2005-3-16")); + data.add(new Tuple2("url_1","2009-12-3")); + data.add(new Tuple2("url_9","2004-4-27")); + data.add(new Tuple2("url_18","2009-5-26")); + data.add(new Tuple2("url_31","2000-9-21")); + data.add(new Tuple2("url_12","2008-9-25")); + data.add(new Tuple2("url_2","2004-2-16")); + data.add(new Tuple2("url_28","2008-11-12")); + data.add(new Tuple2("url_28","2001-6-26")); + data.add(new Tuple2("url_12","2006-3-15")); + data.add(new Tuple2("url_0","2009-3-1")); + data.add(new Tuple2("url_36","2006-10-13")); + data.add(new Tuple2("url_15","2004-11-5")); + data.add(new Tuple2("url_32","2008-2-11")); + data.add(new Tuple2("url_19","2009-8-3")); + data.add(new Tuple2("url_2","2006-8-6")); + data.add(new Tuple2("url_11","2009-10-13")); + data.add(new Tuple2("url_21","2002-9-14")); + data.add(new Tuple2("url_18","2000-11-2")); + data.add(new Tuple2("url_35","2006-5-15")); + data.add(new Tuple2("url_11","2006-2-18")); + data.add(new Tuple2("url_0","2001-4-25")); + data.add(new Tuple2("url_14","2009-4-8")); + data.add(new Tuple2("url_16","2009-4-7")); + + return env.fromCollection(data); + + } + + +} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/generator/WebLogGenerator.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogDataGenerator.java similarity index 67% rename from stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/generator/WebLogGenerator.java rename to stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogDataGenerator.java index 7a3c0236a6462..b60bb4821b440 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/generator/WebLogGenerator.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/util/WebLogDataGenerator.java @@ -11,38 +11,48 @@ * specific language governing permissions and limitations under the License. **********************************************************************************************************************/ -package eu.stratosphere.example.java.relational.generator; +package eu.stratosphere.example.java.relational.util; import java.io.FileWriter; import java.io.IOException; import java.util.Calendar; import java.util.Random; -public class WebLogGenerator { +import eu.stratosphere.example.java.relational.WebLogAnalysis; +/** + * Data generator for the {@link WebLogAnalysis} example program. + * + */ +public class WebLogDataGenerator { + + /** + * Main method to generate data for the {@link WebLogAnalysis} example program. + *

+ * The generator creates to files: + *

    + *
  • {tmp.dir}/documents for the web documents + *
  • {tmp.dir}/ranks for the ranks of the web documents + *
  • {tmp.dir}/visits for the logged visits of web documents + *
+ * + * @param args + *
    + *
  1. Int: Number of web documents + *
  2. Int: Number of visits + *
+ */ public static void main(String[] args) { - if (args.length != 4) { - if (args.length == 0 || args[0].equals("-h") - || args[0].equals("--help")) { - // Show help - System.out.println("Usage:"); - System.out.println("1:\tWith parameters"); - System.out.println("\t " - + "[noDocuments] [noVisits] [outPath] [noFiles]"); - System.out.println("2:\tDefault parameters"); - System.out.println("\t -d"); - return; - } else if (args[0].equals("-d")) { - // Default values - args = new String[4]; - args[0] = "1000"; // number of documents - args[1] = "10000"; // number of visits - args[2] = "/tmp/stratosphere/"; // path - args[3] = "1"; // number of files - } + // parse parameters + if (args.length < 2) { + System.out.println("WebLogDataGenerator "); + System.exit(1); } - + + int noDocs = Integer.parseInt(args[0]); + int noVisits = Integer.parseInt(args[1]); + String[] filterKWs = { "editors", "oscillations", "convection" }; String[] words = { "Lorem", "ipsum", "dolor", "sit", "amet", @@ -53,18 +63,15 @@ public static void main(String[] args) { "ullamcorper", "suscipit", "lobortis", "nisl", "ut", "aliquip", "ex", "ea", "commodo" }; - int noDocs = Integer.parseInt(args[0]); - int noVisits = Integer.parseInt(args[1]); - - String path = args[2]; - int noFiles = Integer.parseInt(args[3]); + + final String outPath = System.getProperty("java.io.tmpdir"); System.out.println("Generating documents files..."); - genDocs(noDocs, noFiles, filterKWs, words, path + "docs_"); + genDocs(noDocs, filterKWs, words, outPath + "/documents"); System.out.println("Generating ranks files..."); - genRanks(noDocs, noFiles, path + "ranks_"); + genRanks(noDocs, outPath + "/ranks"); System.out.println("Generating visits files..."); - genVisits(noVisits, noDocs, noFiles, path + "visits_"); + genVisits(noVisits, noDocs, outPath + "/visits"); System.out.println("Done!"); } @@ -76,8 +83,6 @@ public static void main(String[] args) { * * @param noDocs * Number of entries for the documents relation - * @param noFiles - * Number of files for the documents relation * @param filterKeyWords * A list of keywords that should be contained * @param words @@ -85,16 +90,12 @@ public static void main(String[] args) { * @param path * Output path for the documents relation */ - public static void genDocs(int noDocs, int noFiles, - String[] filterKeyWords, String[] words, String path) { + private static void genDocs(int noDocs, String[] filterKeyWords, String[] words, String path) { Random rand = new Random(Calendar.getInstance().getTimeInMillis()); - int fileId = 0; - int docsPerFile = (noDocs / noFiles) + 1; - int docsInFile = 0; try { - FileWriter fw = new FileWriter(path + (fileId++)); + FileWriter fw = new FileWriter(path); for (int i = 0; i < noDocs; i++) { @@ -102,8 +103,8 @@ public static void genDocs(int noDocs, int noFiles, // URL StringBuilder doc = new StringBuilder("url_" + i + "|"); for (int j = 0; j < wordsInDoc; j++) { - if (rand.nextDouble() > 0.98) { - // Approx. every 50th word is a keyword + if (rand.nextDouble() > 0.9) { + // Approx. every 10th word is a keyword doc.append(filterKeyWords[rand .nextInt(filterKeyWords.length)] + " "); } else { @@ -114,13 +115,6 @@ public static void genDocs(int noDocs, int noFiles, doc.append("|\n"); fw.write(doc.toString()); - - docsInFile++; - if (docsInFile == docsPerFile) { - fw.close(); - fw = new FileWriter(path + (fileId++)); - docsInFile = 0; - } } fw.close(); @@ -136,23 +130,17 @@ public static void genDocs(int noDocs, int noFiles, * * @param noDocs * Number of entries in the documents relation - * @param noFiles - * Number of files for the ranks relation * @param path * Output path for the ranks relation */ - public static void genRanks(int noDocs, int noFiles, String path) { + private static void genRanks(int noDocs, String path) { Random rand = new Random(Calendar.getInstance().getTimeInMillis()); - int fileId = 0; - int docsPerFile = (noDocs / noFiles) + 1; - int docsInFile = 0; try { - FileWriter fw = new FileWriter(path + (fileId++)); + FileWriter fw = new FileWriter(path); for (int i = 0; i < noDocs; i++) { - // Rank StringBuilder rank = new StringBuilder(rand.nextInt(100) + "|"); // URL @@ -161,13 +149,6 @@ public static void genRanks(int noDocs, int noFiles, String path) { rank.append(rand.nextInt(10) + rand.nextInt(50) + "|\n"); fw.write(rank.toString()); - - docsInFile++; - if (docsInFile == docsPerFile) { - fw.close(); - fw = new FileWriter(path + (fileId++)); - docsInFile = 0; - } } fw.close(); @@ -185,20 +166,15 @@ public static void genRanks(int noDocs, int noFiles, String path) { * Number of entries for the visits relation * @param noDocs * Number of entries in the documents relation - * @param noFiles - * Number of files for the visits relation * @param path * Output path for the visits relation */ - public static void genVisits(int noVisits, int noDocs, int noFiles, String path) { + private static void genVisits(int noVisits, int noDocs, String path) { Random rand = new Random(Calendar.getInstance().getTimeInMillis()); - int fileId = 0; - int visitsPerFile = (noVisits / noFiles) + 1; - int visitsInFile = 0; try { - FileWriter fw = new FileWriter(path + (fileId++)); + FileWriter fw = new FileWriter(path); for (int i = 0; i < noVisits; i++) { @@ -218,13 +194,6 @@ public static void genVisits(int noVisits, int noDocs, int noFiles, String path) visit.append("0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n"); fw.write(visit.toString()); - - visitsInFile++; - if (visitsInFile == visitsPerFile) { - fw.close(); - fw = new FileWriter(path + (fileId++)); - visitsInFile = 0; - } } fw.close(); diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java index 17e8819fcc595..47f345baf954e 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java @@ -19,49 +19,64 @@ import eu.stratosphere.api.java.aggregation.Aggregations; import eu.stratosphere.api.java.functions.FlatMapFunction; import eu.stratosphere.api.java.tuple.Tuple2; +import eu.stratosphere.example.java.wordcount.util.WordCountData; import eu.stratosphere.util.Collector; /** * Implements a the "WordCount" program that computes a simple word occurrence histogram - * over text files. The histogram is written back to disk as '(word, count)' pairs. + * over text files. + * + *

+ * The input are plain text files. + * + *

+ * This example shows how to: + *

    + *
  • write a simple Stratosphere program. + *
  • use Tuple data types. + *
  • write and use user-defined functions. + *
+ * */ @SuppressWarnings("serial") public class WordCount { - /** - * Runs the WordCount program. Accepts parameters: . - * Paths must be qualified URIs, i.e., start with "file://..." or "hdfs://...". - * - * @param args Parameters defining the input and output path. - */ + // ************************************************************************* + // PROGRAM + // ************************************************************************* + public static void main(String[] args) throws Exception { - if (args.length < 2) { - System.err.println("Usage: WordCount "); - return; - } - final String inputPath = args[0]; - final String outputPath = args[1]; + parseParameters(args); - // get the environment as starting point + // set up the execution environment final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - // read the text file from given input path - DataSet text = env.readTextFile(inputPath); - - // split up the lines in pairs (2-tuples) containing: (word,1) - DataSet> words = text.flatMap(new Tokenizer()); + // get input data + DataSet text = getTextDataSet(env); - // group by the tuple field "0" and sum up tuple field "1" - DataSet> result = words.groupBy(0).aggregate(Aggregations.SUM, 1); - - // write out the result - result.writeAsText(outputPath); + DataSet> counts = + // split up the lines in pairs (2-tuples) containing: (word,1) + text.flatMap(new Tokenizer()) + // group by the tuple field "0" and sum up tuple field "1" + .groupBy(0) + .aggregate(Aggregations.SUM, 1); + + // emit result + if(fileOutput) { + counts.writeAsCsv(outputPath, "\n", " "); + } else { + counts.print(); + } - // execute the defined program - env.execute("Word Count"); + // execute program + env.execute("WordCount Example"); } + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + /** * Implements the string tokenizer that splits sentences into words as a user-defined * FlatMapFunction. The function takes a line (String) and splits it into @@ -82,4 +97,41 @@ public void flatMap(String value, Collector> out) { } } } + + // ************************************************************************* + // UTIL METHODS + // ************************************************************************* + + private static boolean fileOutput = false; + private static String textPath; + private static String outputPath; + + private static void parseParameters(String[] args) { + + if(args.length > 0) { + // parse input arguments + fileOutput = true; + if(args.length == 2) { + textPath = args[0]; + outputPath = args[1]; + } else { + System.err.println("Usage: WordCount "); + System.exit(1); + } + } else { + System.out.println("Executing WordCount example with built-in default data."); + System.out.println(" Provide parameters to read input data from a file."); + System.out.println(" Usage: WordCount "); + } + } + + private static DataSet getTextDataSet(ExecutionEnvironment env) { + if(fileOutput) { + // read the text file from given input path + return env.readTextFile(textPath); + } else { + // get default test text data + return WordCountData.getDefaultTextLineDataSet(env); + } + } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountCollection.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountCollection.java deleted file mode 100644 index dd9de08c10117..0000000000000 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCountCollection.java +++ /dev/null @@ -1,75 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.example.java.wordcount; - -import static eu.stratosphere.api.java.aggregation.Aggregations.SUM; -import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.api.java.functions.FlatMapFunction; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.util.Collector; - -/** - * Implements the "WordCount" program which takes a collection of strings and counts the number of - * occurrences of each word in these strings. Finally, the result will be written to the - * console. - */ -@SuppressWarnings("serial") -public class WordCountCollection { - - /** - * Runs the WordCount program. - * - * @param args Command line parameters, ignored. - */ - public static void main(String[] args) throws Exception { - - // get the environment as starting point - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // define the strings to be analyzed - DataSet text = env.fromElements("To be", "or not to be", "or to be still", "and certainly not to be not at all", "is that the question?"); - - // split the strings into tuple of (word,1), group by field "0" and sum up field "1" - DataSet> result = text.flatMap(new Tokenizer()).groupBy(0).aggregate(SUM, 1); - - // print the result on the console - result.print(); - - // execute the defined program - env.execute(); - } - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into - * multiple pairs in the form of "(word,1)" (Tuple2). - */ - public static final class Tokenizer extends FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); - } - } - } - } -} diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/util/WordCountData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/util/WordCountData.java new file mode 100644 index 0000000000000..12535bd2d7508 --- /dev/null +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/util/WordCountData.java @@ -0,0 +1,62 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.example.java.wordcount.util; + +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.ExecutionEnvironment; + +public class WordCountData { + + public static DataSet getDefaultTextLineDataSet(ExecutionEnvironment env) { + + return env.fromElements( + "To be, or not to be,--that is the question:--", + "Whether 'tis nobler in the mind to suffer", + "The slings and arrows of outrageous fortune", + "Or to take arms against a sea of troubles,", + "And by opposing end them?--To die,--to sleep,--", + "No more; and by a sleep to say we end", + "The heartache, and the thousand natural shocks", + "That flesh is heir to,--'tis a consummation", + "Devoutly to be wish'd. To die,--to sleep;--", + "To sleep! perchance to dream:--ay, there's the rub;", + "For in that sleep of death what dreams may come,", + "When we have shuffled off this mortal coil,", + "Must give us pause: there's the respect", + "That makes calamity of so long life;", + "For who would bear the whips and scorns of time,", + "The oppressor's wrong, the proud man's contumely,", + "The pangs of despis'd love, the law's delay,", + "The insolence of office, and the spurns", + "That patient merit of the unworthy takes,", + "When he himself might his quietus make", + "With a bare bodkin? who would these fardels bear,", + "To grunt and sweat under a weary life,", + "But that the dread of something after death,--", + "The undiscover'd country, from whose bourn", + "No traveller returns,--puzzles the will,", + "And makes us rather bear those ills we have", + "Than fly to others that we know not of?", + "Thus conscience does make cowards of us all;", + "And thus the native hue of resolution", + "Is sicklied o'er with the pale cast of thought;", + "And enterprises of great pith and moment,", + "With this regard, their currents turn awry,", + "And lose the name of action.--Soft you now!", + "The fair Ophelia!--Nymph, in thy orisons", + "Be all my sins remember'd." + ); + } +} diff --git a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/EnumTriangleData.java b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/EnumTriangleData.java index 8c5eccca3c710..b057fc964b59f 100644 --- a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/EnumTriangleData.java +++ b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/EnumTriangleData.java @@ -17,17 +17,17 @@ public class EnumTriangleData { public static final String EDGES = - "1,2\n" + - "1,3\n" + - "1,4\n" + - "1,5\n" + - "2,3\n" + - "2,5\n" + - "3,4\n" + - "3,7\n" + - "5,6\n" + - "3,8\n" + - "7,8\n"; + "1 2\n" + + "1 3\n" + + "1 4\n" + + "1 5\n" + + "2 3\n" + + "2 5\n" + + "3 4\n" + + "3 7\n" + + "5 6\n" + + "3 8\n" + + "7 8\n"; public static final String TRIANGLES_BY_ID = "1,2,3\n" + diff --git a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/PageRankData.java b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/PageRankData.java index 510bc36eab3a8..4723c69413aeb 100644 --- a/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/PageRankData.java +++ b/stratosphere-test-utils/src/main/java/eu/stratosphere/test/testdata/PageRankData.java @@ -25,35 +25,35 @@ public class PageRankData { "3\n" + "4"; - public static final String VERTICES_WITH_INITIAL_RANK = "1,0.2\n" + - "2,0.2\n" + - "5,0.2\n" + - "3,0.2\n" + - "4,0.2"; + public static final String VERTICES_WITH_INITIAL_RANK = "1 0.2\n" + + "2 0.2\n" + + "5 0.2\n" + + "3 0.2\n" + + "4 0.2"; - public static final String EDGES = "2,1\n" + - "5,2\n" + - "5,4\n" + - "4,3\n" + - "4,2\n" + - "1,4\n" + - "1,2\n" + - "1,3\n" + - "3,5\n"; + public static final String EDGES = "2 1\n" + + "5 2\n" + + "5 4\n" + + "4 3\n" + + "4 2\n" + + "1 4\n" + + "1 2\n" + + "1 3\n" + + "3 5\n"; - public static final String RANKS_AFTER_3_ITERATIONS = "1,0.237\n" + - "2,0.248\n" + - "3,0.173\n" + - "4,0.175\n" + - "5,0.165"; + public static final String RANKS_AFTER_3_ITERATIONS = "1 0.237\n" + + "2 0.248\n" + + "3 0.173\n" + + "4 0.175\n" + + "5 0.165"; - public static final String RANKS_AFTER_EPSILON_0_0001_CONVERGENCE = "1,0.238\n" + - "2,0.244\n" + - "3,0.170\n" + - "4,0.171\n" + - "5,0.174"; + public static final String RANKS_AFTER_EPSILON_0_0001_CONVERGENCE = "1 0.238\n" + + "2 0.244\n" + + "3 0.170\n" + + "4 0.171\n" + + "5 0.174"; private PageRankData() {} } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java index 46df373d32fa9..ff212f83e7db2 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/DumpCompiledPlanTest.java @@ -19,13 +19,10 @@ import org.junit.Test; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.compiler.plan.OptimizedPlan; import eu.stratosphere.compiler.plandump.PlanJSONDumpGenerator; -import eu.stratosphere.example.java.graph.ConnectedComponents; import eu.stratosphere.test.compiler.CompilerTestBase; +import eu.stratosphere.test.recordJobs.graph.DeltaPageRankWithInitialDeltas; import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; @@ -63,16 +60,8 @@ public void dumpBulkIterationKMeans() { } @Test - public void dumpConnectedComponents() { - // take the core program and create dummy sources and sinks around it - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - - DataSet vertices = env.fromElements(1L); - @SuppressWarnings("unchecked") - DataSet> edges = env.fromElements(new Tuple2(1l, 2l)); - - ConnectedComponents.doConnectedComponents(vertices, edges, 10).print(); - dump(env.createProgramPlan()); + public void dumpDeltaPageRank() { + dump(new DeltaPageRankWithInitialDeltas().getPlan(DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, IN_FILE, OUT_FILE, "10")); } private void dump(Plan p) { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java index 112b0cdc3e701..4e8285e3bd8d4 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/compiler/plandump/PreviewPlanDumpTest.java @@ -21,13 +21,10 @@ import org.junit.Test; import eu.stratosphere.api.common.Plan; -import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.ExecutionEnvironment; -import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.compiler.PactCompiler; import eu.stratosphere.compiler.dag.DataSinkNode; import eu.stratosphere.compiler.plandump.PlanJSONDumpGenerator; -import eu.stratosphere.example.java.graph.ConnectedComponents; +import eu.stratosphere.test.recordJobs.graph.DeltaPageRankWithInitialDeltas; import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast; import eu.stratosphere.test.recordJobs.kmeans.KMeansSingleStep; import eu.stratosphere.test.recordJobs.relational.TPCHQuery3; @@ -80,18 +77,9 @@ public void dumpBulkIterationKMeans() { } @Test - public void dumpConnectedComponents() { - // take the core program and create dummy sources and sinks around it - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - - DataSet vertices = env.fromElements(1L); - @SuppressWarnings("unchecked") - DataSet> edges = env.fromElements(new Tuple2(1l, 2l)); - - ConnectedComponents.doConnectedComponents(vertices, edges, 10).print(); - - Plan p = env.createProgramPlan(); - dump(p); + public void dumpDeltaPageRank() { + dump(new DeltaPageRankWithInitialDeltas().getPlan("4", IN_FILE, IN_FILE, IN_FILE, OUT_FILE, "10")); + dump(new DeltaPageRankWithInitialDeltas().getPlan(NO_ARGS)); } private void dump(Plan p) { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleBasicITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleBasicITCase.java index 52b83a6f0ca25..a5bf298f4a616 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleBasicITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleBasicITCase.java @@ -14,7 +14,7 @@ **********************************************************************************************************************/ package eu.stratosphere.test.exampleJavaPrograms; -import eu.stratosphere.example.java.triangles.EnumTrianglesBasic; +import eu.stratosphere.example.java.graph.EnumTrianglesBasic; import eu.stratosphere.test.testdata.EnumTriangleData; import eu.stratosphere.test.util.JavaProgramTestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleOptITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleOptITCase.java index 802d5c2c407c7..90891ff84e55f 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleOptITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/EnumTriangleOptITCase.java @@ -14,7 +14,7 @@ **********************************************************************************************************************/ package eu.stratosphere.test.exampleJavaPrograms; -import eu.stratosphere.example.java.triangles.EnumTrianglesOpt; +import eu.stratosphere.example.java.graph.EnumTrianglesOpt; import eu.stratosphere.test.testdata.EnumTriangleData; import eu.stratosphere.test.util.JavaProgramTestBase; diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/PageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/PageRankITCase.java index 64041e9903939..54c6842e7f1cf 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/PageRankITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/PageRankITCase.java @@ -24,7 +24,7 @@ import org.junit.runners.Parameterized.Parameters; import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.graph.SimplePageRank; +import eu.stratosphere.example.java.graph.PageRankBasic; import eu.stratosphere.test.testdata.PageRankData; import eu.stratosphere.test.util.JavaProgramTestBase; @@ -58,7 +58,7 @@ protected void testProgram() throws Exception { @Override protected void postSubmit() throws Exception { - compareKeyValueParisWithDelta(expectedResult, resultPath, ",", 0.01); + compareKeyValueParisWithDelta(expectedResult, resultPath, " ", 0.01); } @Parameters @@ -80,12 +80,12 @@ public String runProgram(int progId) throws Exception { switch(progId) { case 1: { - SimplePageRank.runPageRank(verticesPath, edgesPath, resultPath, PageRankData.NUM_VERTICES, 3); + PageRankBasic.main(new String[] {verticesPath, edgesPath, resultPath, PageRankData.NUM_VERTICES+"", "3"}); return PageRankData.RANKS_AFTER_3_ITERATIONS; } case 2: { // start with a very high number of iteration such that the dynamic convergence criterion must handle termination - SimplePageRank.runPageRank(verticesPath, edgesPath, resultPath, PageRankData.NUM_VERTICES, 1000); + PageRankBasic.main(new String[] {verticesPath, edgesPath, resultPath, PageRankData.NUM_VERTICES+"", "1000"}); return PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE; } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java index b6c124996ee11..2eaa8a9ddf640 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java @@ -37,7 +37,7 @@ protected void preSubmit() throws Exception { @Override protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(WordCountData.COUNTS_AS_TUPLES, resultPath); + compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath); } @Override diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DeltaPageRankITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DeltaPageRankITCase.java deleted file mode 100644 index f9fb63244cdf6..0000000000000 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DeltaPageRankITCase.java +++ /dev/null @@ -1,150 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.test.javaApiOperators; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.Collection; -import java.util.LinkedList; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.example.java.incremental.pagerank.SimpleDeltaPageRank; -import eu.stratosphere.test.util.JavaProgramTestBase; - -@RunWith(Parameterized.class) -public class DeltaPageRankITCase extends JavaProgramTestBase { - - private static final String INITIAL_VERTICES_WITH_RANK = - "1 0.025\n" + - "2 0.125\n" + - "3 0.0833333333333333\n" + - "4 0.0833333333333333\n" + - "5 0.075\n" + - "6 0.075\n" + - "7 0.183333333333333\n" + - "8 0.15\n" + - "9 0.1\n"; - - private static final String INITIAL_DELTAS = "1 -0.075\n" + - "2 0.025\n" + - "3 -0.0166666666666667\n" + - "4 -0.0166666666666667\n" + - "5 -0.025\n" + - "6 -0.025\n" + - "7 0.0833333333333333\n" + - "8 0.05\n" + - "9 0\n"; - - private static final String EDGES = "1 2 2\n" + - "1 3 2\n" + - "2 3 3\n" + - "2 4 3\n" + - "3 1 4\n" + - "3 2 4\n" + - "4 2 2\n" + - "5 6 2\n" + - "6 5 2\n" + - "7 8 2\n" + - "7 9 2\n" + - "8 7 2\n" + - "8 9 2\n" + - "9 7 2\n" + - "9 8 2\n" + - "3 5 4\n" + - "3 6 4\n" + - "4 8 2\n" + - "2 7 3\n" + - "5 7 2\n" + - "6 4 2\n"; - - private static int NUM_PROGRAMS = 1; - - - private int curProgId = config.getInteger("ProgramId", -1); - - private String resultPath; - protected static String pagesPath; - protected static String edgesPath; - protected static String deltasPath; - private String expectedResult; - - public DeltaPageRankITCase(Configuration config) { - super(config); - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - pagesPath = createTempFile("pages.txt", INITIAL_VERTICES_WITH_RANK); - edgesPath = createTempFile("edges.txt", EDGES); - deltasPath = createTempFile("deltas.txt", INITIAL_DELTAS); - } - - @Override - protected void testProgram() throws Exception { - expectedResult = TestPrograms.runProgram(curProgId, resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expectedResult, resultPath); - } - - @Parameters - public static Collection getConfigurations() throws FileNotFoundException, IOException { - - LinkedList tConfigs = new LinkedList(); - - for(int i=1; i <= NUM_PROGRAMS; i++) { - Configuration config = new Configuration(); - config.setInteger("ProgramId", i); - tConfigs.add(config); - } - - return toParameterList(tConfigs); - } - - private static class TestPrograms { - - public static String runProgram(int progId, String resultPath) throws Exception { - - switch(progId) { - case 1: { - - SimpleDeltaPageRank.run(1, pagesPath, deltasPath, edgesPath, resultPath, 4, false); - - // return expected result - return "1,0.006987847222222211\n" + - "2,0.032682291666666634\n" + - "3,0.018663194444444395\n" + - "4,0.029340277777777726\n" + - "5,0.02209201388888886\n" + - "6,0.02209201388888886\n" + - "7,0.2621527777777774\n" + - "8,0.2607638888888888\n" + - "9,0.2452256944444444\n"; - } - - default: - throw new IllegalArgumentException("Invalid program id"); - } - } - } -} - From 054ebb01880c83ec775aed738e62880db880c54e Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 00:51:42 +0200 Subject: [PATCH 021/182] Updated JAR packaging of Java example programs --- .../stratosphere-java-examples/pom.xml | 226 +++++------------- 1 file changed, 65 insertions(+), 161 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/pom.xml b/stratosphere-examples/stratosphere-java-examples/pom.xml index d586e4d769b78..a2f5ec872bd2e 100644 --- a/stratosphere-examples/stratosphere-java-examples/pom.xml +++ b/stratosphere-examples/stratosphere-java-examples/pom.xml @@ -32,78 +32,6 @@ 2.4 - - - - - TPCHQuery10 - package - - jar - - - TPCHQuery10 - - - - eu.stratosphere.example.java.relational.TPCHQuery10 - - - - **/java/relational/TPCHQuery10.class - **/java/relational/TPCHQuery10$*.class - - - - - - - - - - ConnectedComponents - package - - jar - - - Record-ConnectedComponents - - - - eu.stratosphere.example.java.record.connectedcomponents.WorksetConnectedComponents - - - - - **/record/connectedcomponents/*.class - - - - - - - DanglingPageRank - package - - jar - - - Record-DanglingPageRank - - - - eu.stratosphere.example.java.record.pagerank.DanglingPageRank - - - - - **/record/pagerank/*.class - **/record/util/ConfigUtils.class - - - - KMeans @@ -113,151 +41,146 @@ - Record-KMeans + KMeans - eu.stratosphere.example.java.record.kmeans.KMeans + eu.stratosphere.example.java.clustering.KMeans - **/record/kmeans/KMeans.class - **/record/kmeans/KMeans$*.class - **/record/kmeans/KMeansSampleDataGenerator.class + **/java/clustering/KMeans.class + **/java/clustering/KMeans$*.class + **/java/clustering/util/KMeansDataGenerator.class + **/java/clustering/util/KMeansData.class - + - KMeansIterative + ConnectedComponents package jar - - Record-KMeansIterative + ConnectedComponents - eu.stratosphere.example.java.record.kmeans.KMeansIterative + eu.stratosphere.example.java.graph.ConnectedComponents - **/record/kmeans/KMeansIterative.class - **/record/kmeans/KMeansSampleDataGenerator.class - **/record/kmeans/KMeansIterative$*.class - **/record/kmeans/udfs/*.class + **/java/graph/ConnectedComponents.class + **/java/graph/ConnectedComponents$*.class + **/java/graph/util/ConnectedComponentsData.class - - + + - PairwiseSP + EnumTrianglesBasic package jar - Record-PairwiseSP + EnumTrianglesBasic - eu.stratosphere.example.java.record.shortestpaths.PairwiseSP + eu.stratosphere.example.java.graph.EnumTrianglesBasic - **/record/shortestpaths/PairwiseSP.class - **/record/shortestpaths/PairwiseSP$*.class + **/java/graph/EnumTrianglesBasic.class + **/java/graph/EnumTrianglesBasic$*.class + **/java/graph/util/EnumTrianglesDataTypes.class + **/java/graph/util/EnumTrianglesDataTypes$*.class + **/java/graph/util/EnumTrianglesData.class - - + + - EnumTrianglesOnEdgesWithDegrees + EnumTrianglesOpt package jar - Record-EnumTrianglesOnEdgesWithDegrees + EnumTrianglesOpt - eu.stratosphere.example.java.record.triangles.EnumTrianglesOnEdgesWithDegrees + eu.stratosphere.example.java.graph.EnumTrianglesOpt - **/record/triangles/EnumTrianglesWithDegrees.class - **/record/triangles/EnumTrianglesWithDegrees$*.class - **/record/triangles/ComputeEdgeDegrees.class - **/record/triangles/ComputeEdgeDegrees$*.class - **/record/triangles/EnumTrianglesOnEdgesWithDegrees.class - **/record/triangles/EnumTrianglesOnEdgesWithDegrees$*.class - **/record/triangles/io/*.class + **/java/graph/EnumTrianglesOpt.class + **/java/graph/EnumTrianglesOpt$*.class + **/java/graph/util/EnumTrianglesDataTypes.class + **/java/graph/util/EnumTrianglesDataTypes$*.class + **/java/graph/util/EnumTrianglesData.class - + - EnumTrianglesWithDegrees + PageRankBasic package jar - Record-EnumTrianglesWithDegrees + PageRankBasic - eu.stratosphere.example.java.record.triangles.EnumTrianglesWithDegrees + eu.stratosphere.example.java.graph.PageRankBasic - **/record/triangles/EnumTrianglesWithDegrees.class - **/record/triangles/EnumTrianglesWithDegrees$*.class - **/record/triangles/ComputeEdgeDegrees.class - **/record/triangles/ComputeEdgeDegrees$*.class - **/record/triangles/EnumTrianglesOnEdgesWithDegrees.class - **/record/triangles/EnumTrianglesOnEdgesWithDegrees$*.class - **/record/triangles/io/*.class + **/java/graph/PageRankBasic.class + **/java/graph/PageRankBasic$*.class + **/java/graph/util/PageRankData.class - + - EnumTrianglesRdfFoaf + TPCHQuery10 package jar - Record-EnumTrianglesRdfFoaf + TPCHQuery10 - eu.stratosphere.example.java.record.triangles.EnumTrianglesRdfFoaf + eu.stratosphere.example.java.relational.TPCHQuery10 - - **/record/triangles/EnumTrianglesRdfFoaf.class - **/record/triangles/EnumTrianglesRdfFoaf$*.class + **/java/relational/TPCHQuery10.class + **/java/relational/TPCHQuery10$*.class - - + + TPCHQuery3 package @@ -265,21 +188,20 @@ jar - Record-TPCHQuery3 + TPCHQuery3 - eu.stratosphere.example.java.record.relational.TPCHQuery3 + eu.stratosphere.example.java.relational.TPCHQuery3 - - **/record/relational/TPCHQuery3.class - **/record/relational/TPCHQuery3$*.class + **/java/relational/TPCHQuery3.class + **/java/relational/TPCHQuery3$*.class - + WebLogAnalysis @@ -288,17 +210,19 @@ jar - Record-WebLogAnalysis + WebLogAnalysis - eu.stratosphere.example.java.record.relational.WebLogAnalysis + eu.stratosphere.example.java.relational.WebLogAnalysis - **/record/relational/WebLogAnalysis.class - **/record/relational/WebLogAnalysis$*.class + **/java/relational/WebLogAnalysis.class + **/java/relational/WebLogAnalysis$*.class + **/java/relational/util/WebLogData.class + **/java/relational/util/WebLogDataGenerator.class @@ -311,42 +235,22 @@ jar - Record-WordCount - - - - eu.stratosphere.example.java.record.wordcount.WordCount - - - - - **/record/wordcount/*.class - - - - - - - TeraSort - package - - jar - - - Record-TeraSort + WordCount - eu.stratosphere.example.java.record.sort.TeraSort + eu.stratosphere.example.java.wordcount.WordCount - **/record/sort/TeraSort.class - **/record/sort/terasort/*.class + **/java/wordcount/WordCount.class + **/java/wordcount/WordCount$*.class + **/java/wordcount/util/WordCountData.class + From 0796700c8e2630ba241c0504c2101fd48737ac55 Mon Sep 17 00:00:00 2001 From: vasia Date: Thu, 15 May 2014 12:22:56 +0200 Subject: [PATCH 022/182] small corrections in refactored examples --- .../stratosphere/example/java/clustering/KMeans.java | 8 ++++---- .../example/java/graph/ConnectedComponents.java | 9 +++++---- .../example/java/graph/EnumTrianglesBasic.java | 6 +++--- .../example/java/graph/EnumTrianglesOpt.java | 10 +++++----- .../stratosphere/example/java/graph/PageRankBasic.java | 4 ++-- .../java/graph/util/EnumTrianglesDataTypes.java | 2 +- .../example/java/relational/TPCHQuery3.java | 2 +- .../stratosphere/example/java/wordcount/WordCount.java | 4 ++-- 8 files changed, 23 insertions(+), 22 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java index f0e0f43a142d2..4ebeed87b1503 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java @@ -37,7 +37,7 @@ * Each point is assigned to the cluster center which is closest to it. * Subsequently, each cluster center is moved to the center (mean) of all points that have been assigned to it. * The moved cluster centers are fed into the next iteration. - * The algorithm terminates after a fixed number of iteration (as in this implementation) + * The algorithm terminates after a fixed number of iterations (as in this implementation) * or if cluster centers do not (significantly) move in an iteration. * *

@@ -46,7 +46,7 @@ * each data point is annotated with the id of the final cluster (center) it belongs to. * *

- * Input files are plain text files must be formatted as follows: + * Input files are plain text files and must be formatted as follows: *

    *
  • Data points are represented as two double values separated by a blank character. * Data points are separated by newline characters.
    @@ -84,7 +84,7 @@ public static void main(String[] args) throws Exception { // set number of bulk iterations for KMeans algorithm IterativeDataSet loop = centroids.iterate(numIterations); - DataSet newCentriods = points + DataSet newCentroids = points // compute closest centroid for each point .map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids") // count and sum point coordinates for each centroid @@ -94,7 +94,7 @@ public static void main(String[] args) throws Exception { .map(new CentroidAverager()); // feed new centroids back into next iteration - DataSet finalCentroids = loop.closeWith(newCentriods); + DataSet finalCentroids = loop.closeWith(newCentroids); DataSet> clusteredPoints = points // assign points to final clusters diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java index 29c55f210bfec..a2f402a922b17 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java @@ -29,7 +29,8 @@ /** * An implementation of the connected components algorithm, using a delta iteration. * Initially, the algorithm assigns each vertex its own ID. After the algorithm has completed, all vertices in the - * same component will have the same id. In each step, a vertex + * same component will have the same id. In each step, a vertex picks the minimum of its own ID and its + * neighbors' IDs, as its new ID. *

    * A vertex whose component did not change needs not propagate its information in the next step. Because of that, * the algorithm is easily expressible via a delta iteration. We here model the solution set as the vertices with @@ -38,9 +39,9 @@ * is consequently also the next workset. * *

    - * Input files are plain text files must be formatted as follows: + * Input files are plain text files and must be formatted as follows: *

      - *
    • Vertexes represented as IDs and separated by new-line characters.
      + *
    • Vertices represented as IDs and separated by new-line characters.
      * For example "1\n2\n12\n42\n63\n" gives five vertices (1), (2), (12), (42), and (63). *
    • Edges are represented as pairs for vertex IDs which are separated by space * characters. Edges are separated by new-line characters.
      @@ -79,7 +80,7 @@ public static void main(String... args) throws Exception { DeltaIteration, Tuple2> iteration = verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0); - // apply the step logic: join with the edges, select the minimum neighbor, update the component of the candidate is smaller + // apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller DataSet> changes = iteration.getWorkset().join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin()) .groupBy(0).aggregate(Aggregations.MIN, 1) .join(iteration.getSolutionSet()).where(0).equalTo(0) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java index 6754f33a33d2e..4337fd48e5ae1 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java @@ -32,7 +32,7 @@ /** * Triangle enumeration is a preprocessing step to find closely connected parts in graphs. - * A triangle are three edges that connect three vertices with each other. + * A triangle consists of three edges that connect three vertices with each other. * *

      * The algorithm works as follows: @@ -41,7 +41,7 @@ * that closes the triangle. * *

      - * Input files are plain text files must be formatted as follows: + * Input files are plain text files and must be formatted as follows: *

        *
      • Edges are represented as pairs for vertex IDs which are separated by space * characters. Edges are separated by new-line characters.
        @@ -128,7 +128,7 @@ public Edge map(Edge inEdge) throws Exception { // flip vertices if necessary if(inEdge.getFirstVertex() > inEdge.getSecondVertex()) { - inEdge.flipVertics(); + inEdge.flipVertices(); } return inEdge; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java index 986fd9eaad8a0..57ff558bf2b8b 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java @@ -35,7 +35,7 @@ /** * Triangle enumeration is a preprocessing step to find closely connected parts in graphs. - * A triangle are three edges that connect three vertices with each other. + * A triangle consists of three edges that connect three vertices with each other. * *

        * The basic algorithm works as follows: @@ -51,7 +51,7 @@ * grouping on edges on the vertex with the smaller degree. * *

        - * Input files are plain text files must be formatted as follows: + * Input files are plain text files and must be formatted as follows: *

          *
        • Edges are represented as pairs for vertex IDs which are separated by space * characters. Edges are separated by new-line characters.
          @@ -141,7 +141,7 @@ private static class EdgeDuplicator extends FlatMapFunction { @Override public void flatMap(Edge edge, Collector out) throws Exception { out.collect(edge); - edge.flipVertics(); + edge.flipVertices(); out.collect(edge); } } @@ -231,7 +231,7 @@ public Edge map(EdgeWithDegrees inEdge) throws Exception { // flip vertices if first degree is larger than second degree. if(inEdge.getFirstDegree() > inEdge.getSecondDegree()) { - outEdge.flipVertics(); + outEdge.flipVertices(); } // return edge @@ -247,7 +247,7 @@ public Edge map(Edge inEdge) throws Exception { // flip vertices if necessary if(inEdge.getFirstVertex() > inEdge.getSecondVertex()) { - inEdge.flipVertics(); + inEdge.flipVertices(); } return inEdge; diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java index b3deb67abaa80..cf66cce012bb9 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java @@ -37,12 +37,12 @@ * This implementation requires a set of pages (vertices) with associated ranks and a set * of directed links (edges) as input and works as follows.
          * In each iteration, the rank of every page is evenly distributed to all pages it points to. - * Each page collects the partial ranks of all pages that point to it, sums them up, and apply a dampening factor to the sum. + * Each page collects the partial ranks of all pages that point to it, sums them up, and applies a dampening factor to the sum. * The result is the new rank of the page. A new iteration is started with the new ranks of all pages. * This implementation terminates after a fixed number of iterations. * *

          - * Input files are plain text files must be formatted as follows: + * Input files are plain text files and must be formatted as follows: *

            *
          • Pages represented as an (long) ID and a (double) rank separated by new-line characters.
            * For example "1 0.4\n2 0.3\n12 0.15\n42 0.05\n63 0.1\n" gives five pages with associated ranks diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java index 39719857b5b2c..81489ce39b503 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/util/EnumTrianglesDataTypes.java @@ -52,7 +52,7 @@ public void copyVerticesFromEdgeWithDegrees(EdgeWithDegrees ewd) { this.setSecondVertex(ewd.getSecondVertex()); } - public void flipVertics() { + public void flipVertices() { Integer tmp = this.getFirstVertex(); this.setFirstVertex(this.getSecondVertex()); this.setSecondVertex(tmp); diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java index 794eb2d4c2feb..7b86c3bd95ee5 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java @@ -157,7 +157,7 @@ public ShippingPriorityItem join(Customer first, Order second) { } }); - // Join the last join result with Orders + // Join the last join result with LineItems DataSet joined = customerWithOrders.join(li) .where(4) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java index 47f345baf954e..40c2f21d73d35 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java @@ -23,11 +23,11 @@ import eu.stratosphere.util.Collector; /** - * Implements a the "WordCount" program that computes a simple word occurrence histogram + * Implements the "WordCount" program that computes a simple word occurrence histogram * over text files. * *

            - * The input are plain text files. + * The input is plain text files. * *

            * This example shows how to: From 10d62283ba639a2d7738dc18581606b39a343232 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 14 May 2014 19:54:27 +0200 Subject: [PATCH 023/182] ValueComparator uses ascending flag, tests --- .../runtime/CopyableValueComparator.java | 9 ++-- .../typeutils/runtime/ValueComparator.java | 9 ++-- .../runtime/CopyableValueComparatorTest.java | 49 +++++++++++++++++ .../runtime/ValueComparatorTest.java | 54 +++++++++++++++++++ 4 files changed, 115 insertions(+), 6 deletions(-) create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparator.java index 6202bafae8d6f..ec4e0e5dd6a12 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparator.java @@ -64,12 +64,14 @@ public boolean equalToReference(T candidate) { @Override public int compareToReference(TypeComparator referencedComparator) { T otherRef = ((CopyableValueComparator) referencedComparator).reference; - return otherRef.compareTo(reference); + int comp = otherRef.compareTo(reference); + return ascendingComparison ? comp : -comp; } @Override public int compare(T first, T second) { - return first.compareTo(second); + int comp = first.compareTo(second); + return ascendingComparison ? comp : -comp; } @Override @@ -80,7 +82,8 @@ public int compare(DataInputView firstSource, DataInputView secondSource) throws reference.read(firstSource); tempReference.read(secondSource); - return reference.compareTo(tempReference); + int comp = reference.compareTo(tempReference); + return ascendingComparison ? comp : -comp; } @Override diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparator.java index 9b048dfe0731b..635d064dedade 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparator.java @@ -68,12 +68,14 @@ public boolean equalToReference(T candidate) { @Override public int compareToReference(TypeComparator referencedComparator) { T otherRef = ((ValueComparator) referencedComparator).reference; - return otherRef.compareTo(reference); + int comp = otherRef.compareTo(reference); + return ascendingComparison ? comp : -comp; } @Override public int compare(T first, T second) { - return first.compareTo(second); + int comp = first.compareTo(second); + return ascendingComparison ? comp : -comp; } @Override @@ -87,7 +89,8 @@ public int compare(DataInputView firstSource, DataInputView secondSource) throws reference.read(firstSource); tempReference.read(secondSource); - return reference.compareTo(tempReference); + int comp = reference.compareTo(tempReference); + return ascendingComparison ? comp : -comp; } @Override diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java new file mode 100644 index 0000000000000..8a13571577d17 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import eu.stratosphere.api.common.typeutils.ComparatorTestBase; +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.TypeSerializer; +import eu.stratosphere.types.StringValue; + +public class CopyableValueComparatorTest extends ComparatorTestBase { + + StringValue[] data = new StringValue[]{ + new StringValue(""), + new StringValue("Lorem Ipsum Dolor Omit Longer"), + new StringValue("aaaa"), + new StringValue("abcd"), + new StringValue("abce"), + new StringValue("abdd"), + new StringValue("accd"), + new StringValue("bbcd") + }; + + @Override + protected TypeComparator createComparator(boolean ascending) { + return new CopyableValueComparator(ascending, StringValue.class); + } + + @Override + protected TypeSerializer createSerializer() { + return new CopyableValueSerializer(StringValue.class); + } + + @Override + protected StringValue[] getSortedTestData() { + return data; + } +} diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java new file mode 100644 index 0000000000000..d00d4c1bae81f --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java @@ -0,0 +1,54 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import eu.stratosphere.api.common.typeutils.ComparatorTestBase; +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.TypeSerializer; +import eu.stratosphere.api.common.typeutils.base.StringValueSerializer; +import eu.stratosphere.types.StringValue; +import eu.stratosphere.types.Value; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class ValueComparatorTest extends ComparatorTestBase { + + StringValue[] data = new StringValue[]{ + new StringValue(""), + new StringValue("Lorem Ipsum Dolor Omit Longer"), + new StringValue("aaaa"), + new StringValue("abcd"), + new StringValue("abce"), + new StringValue("abdd"), + new StringValue("accd"), + new StringValue("bbcd") + }; + + @Override + protected TypeComparator createComparator(boolean ascending) { + return new ValueComparator(ascending, StringValue.class); + } + + @Override + protected TypeSerializer createSerializer() { + return new ValueSerializer(StringValue.class); + } + + @Override + protected StringValue[] getSortedTestData() { + return data; + } +} From eb2bc03a1cb3aa1851cdb8ab365b3b8d78feacaf Mon Sep 17 00:00:00 2001 From: uce Date: Wed, 14 May 2014 20:27:58 +0200 Subject: [PATCH 024/182] Add GenericTypeComparator --- .../common/typeutils/ComparatorTestBase.java | 9 +- .../typeutils/base/ByteComparatorTest.java | 2 - .../api/java/typeutils/GenericTypeInfo.java | 18 +- .../runtime/GenericTypeComparator.java | 178 +++++++++ .../RuntimeStatefulSerializerFactory.java | 4 +- .../runtime/GenericArraySerializerTest.java | 10 +- .../runtime/GenericTypeComparatorTest.java | 371 ++++++++++++++++++ ...st.java => GenericTypeSerializerTest.java} | 115 +++--- .../runtime/TupleSerializerTest.java | 10 +- 9 files changed, 640 insertions(+), 77 deletions(-) create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparator.java create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java rename stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/{GenericSerializerTest.java => GenericTypeSerializerTest.java} (96%) diff --git a/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/ComparatorTestBase.java b/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/ComparatorTestBase.java index fef71ba2a39d9..c18f632af48d6 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/ComparatorTestBase.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/ComparatorTestBase.java @@ -43,6 +43,13 @@ public abstract class ComparatorTestBase { protected abstract TypeSerializer createSerializer(); + /** + * Returns the sorted data set. + *

            + * Note: every element needs to be *strictly greater* than the previous element. + * + * @return sorted test data set + */ protected abstract T[] getSortedTestData(); // -------------------------------- test duplication ------------------------------------------ @@ -433,7 +440,7 @@ protected void writeSortedData(T value, TestOutputView out) throws IOException { assertTrue("No data available during deserialization.", in.available() > 0); T deserialized = serializer.deserialize(serializer.createInstance(), in); - deepEquals("Deserialized value if wrong.", value, deserialized); + deepEquals("Deserialized value is wrong.", value, deserialized); } diff --git a/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/base/ByteComparatorTest.java b/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/base/ByteComparatorTest.java index 1c41fc2b3482f..bce5d51e9e4fd 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/base/ByteComparatorTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/api/common/typeutils/base/ByteComparatorTest.java @@ -17,8 +17,6 @@ import eu.stratosphere.api.common.typeutils.ComparatorTestBase; import eu.stratosphere.api.common.typeutils.TypeComparator; import eu.stratosphere.api.common.typeutils.TypeSerializer; -import eu.stratosphere.api.common.typeutils.base.ByteComparator; -import eu.stratosphere.api.common.typeutils.base.ByteSerializer; import java.util.Random; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/GenericTypeInfo.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/GenericTypeInfo.java index 9f2f61336b8ed..1cf4439ed9c0f 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/GenericTypeInfo.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/GenericTypeInfo.java @@ -17,6 +17,7 @@ import eu.stratosphere.api.common.typeutils.TypeComparator; import eu.stratosphere.api.common.typeutils.TypeSerializer; import eu.stratosphere.api.java.typeutils.runtime.AvroSerializer; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeComparator; /** @@ -60,19 +61,26 @@ public boolean isKeyType() { public TypeSerializer createSerializer() { return new AvroSerializer(this.typeClass); } - + @Override public TypeComparator createComparator(boolean sortOrderAscending) { - throw new UnsupportedOperationException("Generic type comparators are not yet implemented."); + if (isKeyType()) { + @SuppressWarnings("unchecked") + GenericTypeComparator comparator = new GenericTypeComparator(sortOrderAscending, createSerializer(), this.typeClass); + + return comparator; + } + + throw new UnsupportedOperationException("Generic types that don't implement java.lang.Comparable cannot be used as keys."); } - + // -------------------------------------------------------------------------------------------- - + @Override public int hashCode() { return typeClass.hashCode() ^ 0x165667b1; } - + @Override public boolean equals(Object obj) { if (obj.getClass() == GenericTypeInfo.class) { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparator.java new file mode 100644 index 0000000000000..861f99441f606 --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparator.java @@ -0,0 +1,178 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import com.esotericsoftware.kryo.Kryo; +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.TypeSerializer; +import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; +import eu.stratosphere.core.memory.MemorySegment; +import eu.stratosphere.types.NormalizableKey; +import eu.stratosphere.util.InstantiationUtil; + +import java.io.IOException; + +/** + * TypeComparator for all types that extend Comparable. + */ +public class GenericTypeComparator> extends TypeComparator { + + private static final long serialVersionUID = 1L; + + private final boolean ascending; + + private final Class type; + + private final TypeSerializerFactory serializerFactory; + + private transient TypeSerializer serializer; + + private transient T reference; + + private transient T tmpReference; + + private transient Kryo kryo; + + // ------------------------------------------------------------------------ + + public GenericTypeComparator(boolean ascending, TypeSerializer serializer, Class type) { + this.ascending = ascending; + this.serializer = serializer; + this.type = type; + + this.serializerFactory = this.serializer.isStateful() + ? new RuntimeStatefulSerializerFactory(this.serializer, this.type) + : new RuntimeStatelessSerializerFactory(this.serializer, this.type); + } + + private GenericTypeComparator(GenericTypeComparator toClone) { + this.ascending = toClone.ascending; + this.serializerFactory = toClone.serializerFactory; + this.type = toClone.type; + } + + @Override + public int hash(T record) { + return record.hashCode(); + } + + @Override + public void setReference(T toCompare) { + checkKryoInitialized(); + this.reference = this.kryo.copy(toCompare); + } + + @Override + public boolean equalToReference(T candidate) { + return candidate.equals(this.reference); + } + + @Override + public int compareToReference(TypeComparator referencedComparator) { + T otherRef = ((GenericTypeComparator) referencedComparator).reference; + int cmp = otherRef.compareTo(this.reference); + + return this.ascending ? cmp : -cmp; + } + + @Override + public int compare(T first, T second) { + return first.compareTo(second); + } + + @Override + public int compare(final DataInputView firstSource, final DataInputView secondSource) throws IOException { + if (this.serializer == null) { + this.serializer = this.serializerFactory.getSerializer(); + } + + if (this.reference == null) { + this.reference = this.serializer.createInstance(); + } + + if (this.tmpReference == null) { + this.tmpReference = this.serializer.createInstance(); + } + + this.reference = this.serializer.deserialize(this.reference, firstSource); + this.tmpReference = this.serializer.deserialize(this.tmpReference, secondSource); + + int cmp = this.reference.compareTo(this.tmpReference); + return this.ascending ? cmp : -cmp; + } + + @Override + public boolean supportsNormalizedKey() { + return NormalizableKey.class.isAssignableFrom(this.type); + } + + @Override + public int getNormalizeKeyLen() { + if (this.reference == null) { + this.reference = InstantiationUtil.instantiate(this.type); + } + + NormalizableKey key = (NormalizableKey) this.reference; + return key.getMaxNormalizedKeyLen(); + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return keyBytes < getNormalizeKeyLen(); + } + + @Override + public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { + NormalizableKey key = (NormalizableKey) record; + key.copyNormalizedKey(target, offset, numBytes); + } + + @Override + public boolean invertNormalizedKey() { + return !ascending; + } + + @Override + public TypeComparator duplicate() { + return new GenericTypeComparator(this); + } + + private final void checkKryoInitialized() { + if (this.kryo == null) { + this.kryo = new Kryo(); + this.kryo.setAsmEnabled(true); + this.kryo.register(this.type); + } + } + + // ------------------------------------------------------------------------ + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException(); + } +} diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/RuntimeStatefulSerializerFactory.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/RuntimeStatefulSerializerFactory.java index fb7f3ff9f3611..78bd3b8340ddb 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/RuntimeStatefulSerializerFactory.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/RuntimeStatefulSerializerFactory.java @@ -23,8 +23,7 @@ public final class RuntimeStatefulSerializerFactory implements TypeSerializerFactory, java.io.Serializable { private static final long serialVersionUID = 1L; - - + private static final String CONFIG_KEY_SER = "SER_DATA"; private static final String CONFIG_KEY_CLASS = "CLASS_DATA"; @@ -37,7 +36,6 @@ public final class RuntimeStatefulSerializerFactory implements TypeSerializer private Class clazz; - public RuntimeStatefulSerializerFactory() {} public RuntimeStatefulSerializerFactory(TypeSerializer serializer, Class clazz) { diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericArraySerializerTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericArraySerializerTest.java index f2e248090ec89..891df0d097121 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericArraySerializerTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericArraySerializerTest.java @@ -25,11 +25,11 @@ import eu.stratosphere.api.common.typeutils.SerializerTestInstance; import eu.stratosphere.api.common.typeutils.TypeSerializer; import eu.stratosphere.api.common.typeutils.base.StringSerializer; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.Book; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.BookAuthor; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.ComplexNestedObject1; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.ComplexNestedObject2; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.SimpleTypes; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.Book; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.BookAuthor; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.ComplexNestedObject1; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.ComplexNestedObject2; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.SimpleTypes; import eu.stratosphere.util.StringUtils; public class GenericArraySerializerTest { diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java new file mode 100644 index 0000000000000..2b8271cd282ec --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java @@ -0,0 +1,371 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import eu.stratosphere.api.common.typeutils.ComparatorTestBase; +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.TypeSerializer; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class GenericTypeComparatorTest { + + @Test + public void testString() { + runTests(new String[]{ + "", + "Lorem Ipsum Dolor Omit Longer", + "aaaa", + "abcd", + "abce", + "abdd", + "accd", + "bbcd" + }); + } + + @Test + public void testSimpleTypesObjects() { + runTests( + new SimpleTypes(0, 1, (byte) 2, "", (short) 3, 4.0), + new SimpleTypes(1, 1, (byte) 2, "", (short) 3, 4.0), + new SimpleTypes(1, 2, (byte) 2, "", (short) 3, 4.0), + new SimpleTypes(1, 2, (byte) 3, "", (short) 3, 4.0), + new SimpleTypes(1, 2, (byte) 3, "a", (short) 3, 4.0), + new SimpleTypes(1, 2, (byte) 3, "b", (short) 3, 4.0), + new SimpleTypes(1, 2, (byte) 3, "b", (short) 4, 4.0), + new SimpleTypes(1, 2, (byte) 3, "b", (short) 4, 6.0) + ); + } + + @Test + public void testCompositeObject() { + ComplexNestedObject1 o1 = new ComplexNestedObject1(-1100); + ComplexNestedObject1 o2 = new ComplexNestedObject1(0); + ComplexNestedObject1 o3 = new ComplexNestedObject1(44); + ComplexNestedObject1 o4 = new ComplexNestedObject1(76923, "A"); + ComplexNestedObject1 o5 = new ComplexNestedObject1(5626435, "A somewhat random collection"); + + runTests(o1, o2, o3, o4, o5); + } + + @Test + public void testBeanStyleObjects() { + { + Book b111 = new Book(-1L, "A Low level interfaces", 0xC); + Book b122 = new Book(-1L, "Low level interfaces", 0xC); + Book b123 = new Book(-1L, "Low level interfaces", 0xC0FFEE); + + Book b2 = new Book(0L, "Debugging byte streams", 1337); + Book b3 = new Book(976243875L, "The Serialization Odysse", 42); + + runTests(b111, b122, b123, b2, b3); + } + + { + BookAuthor b1 = new BookAuthor(976243875L, new ArrayList(), "Arno Nym"); + + ArrayList list = new ArrayList(); + list.add("A"); + list.add("B"); + list.add("C"); + list.add("D"); + list.add("E"); + + BookAuthor b2 = new BookAuthor(976243875L, list, "The Saurus"); + + runTests(b1, b2); + } + } + + // ------------------------------------------------------------------------ + + private final void runTests(T... sortedTestData) { + ComparatorTestInstance testBase = new ComparatorTestInstance(sortedTestData); + testBase.testAll(); + } + + private static final TypeSerializer createSerializer(Class type) { + return new AvroSerializer(type); + } + + // ------------------------------------------------------------------------ + // test instance + // ------------------------------------------------------------------------ + + private class ComparatorTestInstance extends ComparatorTestBase { + + private final T[] testData; + + private final Class type; + + public ComparatorTestInstance(T[] testData) { + if (testData == null || testData.length == 0) { + throw new IllegalArgumentException(); + } + + this.testData = testData; + this.type = (Class) testData[0].getClass(); + } + + @Override + protected TypeComparator createComparator(boolean ascending) { + return new GenericTypeComparator(ascending, GenericTypeComparatorTest.createSerializer(this.type), this.type); + } + + @Override + protected TypeSerializer createSerializer() { + return GenericTypeComparatorTest.createSerializer(this.type); + } + + @Override + protected T[] getSortedTestData() { + return this.testData; + } + + public void testAll() { + testDuplicate(); + testEquality(); + testEqualityWithReference(); + testInequality(); + testInequalityWithReference(); + testNormalizedKeysEqualsFullLength(); + testNormalizedKeysEqualsHalfLength(); + testNormalizedKeysGreatSmallFullLength(); + testNormalizedKeysGreatSmallAscDescHalfLength(); + testNormalizedKeyReadWriter(); + } + } + + // ------------------------------------------------------------------------ + // test objects + // ------------------------------------------------------------------------ + + public static final class SimpleTypes implements Comparable { + + private final int iVal; + private final long lVal; + private final byte bVal; + private final String sVal; + private final short rVal; + private final double dVal; + + public SimpleTypes() { + this(0, 0, (byte) 0, "", (short) 0, 0); + } + + public SimpleTypes(int iVal, long lVal, byte bVal, String sVal, short rVal, double dVal) { + this.iVal = iVal; + this.lVal = lVal; + this.bVal = bVal; + this.sVal = sVal; + this.rVal = rVal; + this.dVal = dVal; + } + + @Override + public String toString() { + return String.format("(%d, %d, %d, %s, %d, %f)", iVal, lVal, bVal, sVal, rVal, dVal); + } + + @Override + public boolean equals(Object obj) { + if (obj.getClass() == SimpleTypes.class) { + SimpleTypes other = (SimpleTypes) obj; + + return other.iVal == this.iVal && + other.lVal == this.lVal && + other.bVal == this.bVal && + other.sVal.equals(this.sVal) && + other.rVal == this.rVal && + other.dVal == this.dVal; + } else { + return false; + } + } + + @Override + public int compareTo(SimpleTypes o) { + int cmp = (this.iVal < o.iVal ? -1 : (this.iVal == o.iVal ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + cmp = (this.lVal < o.lVal ? -1 : (this.lVal == o.lVal ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + cmp = (this.bVal < o.bVal ? -1 : (this.bVal == o.bVal ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + cmp = this.sVal.compareTo(o.sVal); + if (cmp != 0) { + return cmp; + } + + cmp = (this.rVal < o.rVal ? -1 : (this.rVal == o.rVal ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + return (this.dVal < o.dVal ? -1 : (this.dVal == o.dVal ? 0 : 1)); + } + } + + public static class ComplexNestedObject1 implements Comparable { + + private double doubleValue; + + private List stringList; + + public ComplexNestedObject1() { + } + + public ComplexNestedObject1(double value, String... listElements) { + this.doubleValue = value; + + this.stringList = new ArrayList(); + for (String str : listElements) { + this.stringList.add(str); + } + } + + @Override + public boolean equals(Object obj) { + if (obj.getClass() == ComplexNestedObject1.class) { + ComplexNestedObject1 other = (ComplexNestedObject1) obj; + return other.doubleValue == this.doubleValue && this.stringList.equals(other.stringList); + } else { + return false; + } + } + + @Override + public int compareTo(ComplexNestedObject1 o) { + int cmp = (this.doubleValue < o.doubleValue ? -1 : (this.doubleValue == o.doubleValue ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + int size = this.stringList.size(); + int otherSize = o.stringList.size(); + + cmp = (size < otherSize ? -1 : (size == otherSize ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + for (int i = 0; i < size; i++) { + cmp = this.stringList.get(i).compareTo(o.stringList.get(i)); + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + } + + public static class Book implements Comparable { + + private long bookId; + private String title; + private long authorId; + + public Book() { + } + + public Book(long bookId, String title, long authorId) { + this.bookId = bookId; + this.title = title; + this.authorId = authorId; + } + + @Override + public boolean equals(Object obj) { + if (obj.getClass() == Book.class) { + Book other = (Book) obj; + return other.bookId == this.bookId && other.authorId == this.authorId && this.title.equals(other.title); + } else { + return false; + } + } + + @Override + public int compareTo(Book o) { + int cmp = (this.bookId < o.bookId ? -1 : (this.bookId == o.bookId ? 0 : 1)); + if (cmp != 0) { + return cmp; + } + + cmp = title.compareTo(o.title); + if (cmp != 0) { + return cmp; + } + + return (this.authorId < o.authorId ? -1 : (this.authorId == o.authorId ? 0 : 1)); + } + } + + public static class BookAuthor implements Comparable { + + private long authorId; + private List bookTitles; + private String authorName; + + public BookAuthor() { + } + + public BookAuthor(long authorId, List bookTitles, String authorName) { + this.authorId = authorId; + this.bookTitles = bookTitles; + this.authorName = authorName; + } + + @Override + public boolean equals(Object obj) { + if (obj.getClass() == BookAuthor.class) { + BookAuthor other = (BookAuthor) obj; + return other.authorName.equals(this.authorName) && other.authorId == this.authorId && + other.bookTitles.equals(this.bookTitles); + } else { + return false; + } + } + + @Override + public int compareTo(BookAuthor o) { + int cmp = (this.authorId < o.authorId ? -1 : (this.authorId == o.authorId ? 0 : 1)); + if (cmp != 0) return cmp; + + int size = this.bookTitles.size(); + int oSize = o.bookTitles.size(); + cmp = (size < oSize ? -1 : (size == oSize ? 0 : 1)); + if (cmp != 0) return cmp; + + for (int i = 0; i < size; i++) { + cmp = this.bookTitles.get(i).compareTo(o.bookTitles.get(i)); + if (cmp != 0) return cmp; + } + + return this.authorName.compareTo(o.authorName); + } + } +} diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericSerializerTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeSerializerTest.java similarity index 96% rename from stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericSerializerTest.java rename to stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeSerializerTest.java index cce1b613e1d59..a6809bf9a1930 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericSerializerTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeSerializerTest.java @@ -14,34 +14,33 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.typeutils.runtime; +import eu.stratosphere.api.common.typeutils.SerializerTestInstance; +import eu.stratosphere.util.StringUtils; +import org.junit.Test; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; -import org.junit.Test; - -import eu.stratosphere.api.common.typeutils.SerializerTestInstance; -import eu.stratosphere.util.StringUtils; - /** * A test for the {@link AvroSerializer}. */ -public class GenericSerializerTest { +public class GenericTypeSerializerTest { private final Random rnd = new Random(349712539451944123L); - - + + @Test public void testString() { runTests("abc", "", StringUtils.getRandomString(new Random(289347567856686223L), 10, 100), StringUtils.getRandomString(new Random(289347567856686223L), 1000, 5000), StringUtils.getRandomString(new Random(289347567856686223L), 30000, 35000), - StringUtils.getRandomString(new Random(289347567856686223L), 100*1024, 105*1024)); + StringUtils.getRandomString(new Random(289347567856686223L), 100 * 1024, 105 * 1024)); } - + @Test public void testSimpleTypesObjects() { SimpleTypes a = new SimpleTypes(); @@ -57,10 +56,10 @@ public void testSimpleTypesObjects() { StringUtils.getRandomString(rnd, 10, 100), (short) rnd.nextInt(), rnd.nextDouble()); SimpleTypes g = new SimpleTypes(rnd.nextInt(), rnd.nextLong(), (byte) rnd.nextInt(), StringUtils.getRandomString(rnd, 10, 100), (short) rnd.nextInt(), rnd.nextDouble()); - + runTests(a, b, c, d, e, f, g); } - + @Test public void testCompositeObject() { ComplexNestedObject1 o1 = new ComplexNestedObject1(5626435); @@ -68,30 +67,30 @@ public void testCompositeObject() { ComplexNestedObject1 o3 = new ComplexNestedObject1(-1100); ComplexNestedObject1 o4 = new ComplexNestedObject1(0); ComplexNestedObject1 o5 = new ComplexNestedObject1(44); - + runTests(o1, o2, o3, o4, o5); } - + @Test public void testNestedObjects() { ComplexNestedObject2 o1 = new ComplexNestedObject2(rnd); ComplexNestedObject2 o2 = new ComplexNestedObject2(); ComplexNestedObject2 o3 = new ComplexNestedObject2(rnd); ComplexNestedObject2 o4 = new ComplexNestedObject2(rnd); - + runTests(o1, o2, o3, o4); } - + @Test public void testBeanStyleObjects() { { Book b1 = new Book(976243875L, "The Serialization Odysse", 42); Book b2 = new Book(0L, "Debugging byte streams", 1337); Book b3 = new Book(-1L, "Low level interfaces", 0xC0FFEE); - + runTests(b1, b2, b3); } - + // object with collection { ArrayList list = new ArrayList(); @@ -100,52 +99,52 @@ public void testBeanStyleObjects() { list.add("C"); list.add("D"); list.add("E"); - + BookAuthor b1 = new BookAuthor(976243875L, list, "Arno Nym"); - + ArrayList list2 = new ArrayList(); BookAuthor b2 = new BookAuthor(987654321L, list2, "The Saurus"); - + runTests(b1, b2); } } - - private final void runTests(T... instances) { + + private final void runTests(T... instances) { if (instances == null || instances.length == 0) { throw new IllegalArgumentException(); } - + @SuppressWarnings("unchecked") Class clazz = (Class) instances[0].getClass(); - + AvroSerializer serializer = createSerializer(clazz); SerializerTestInstance test = new SerializerTestInstance(serializer, clazz, -1, instances); test.testAll(); } - + private final AvroSerializer createSerializer(Class type) { return new AvroSerializer(type); } - + // -------------------------------------------------------------------------------------------- // Test Objects // -------------------------------------------------------------------------------------------- public static final class SimpleTypes { - + private final int iVal; private final long lVal; private final byte bVal; private final String sVal; private final short rVal; private final double dVal; - - + + public SimpleTypes() { this(0, 0, (byte) 0, "", (short) 0, 0); } - + public SimpleTypes(int iVal, long lVal, byte bVal, String sVal, short rVal, double dVal) { this.iVal = iVal; this.lVal = lVal; @@ -154,41 +153,42 @@ public SimpleTypes(int iVal, long lVal, byte bVal, String sVal, short rVal, doub this.rVal = rVal; this.dVal = dVal; } - + @Override public boolean equals(Object obj) { if (obj.getClass() == SimpleTypes.class) { SimpleTypes other = (SimpleTypes) obj; - + return other.iVal == this.iVal && other.lVal == this.lVal && other.bVal == this.bVal && other.sVal.equals(this.sVal) && other.rVal == this.rVal && other.dVal == this.dVal; - + } else { return false; } } - + @Override public String toString() { return String.format("(%d, %d, %d, %s, %d, %f)", iVal, lVal, bVal, sVal, rVal, dVal); } } - + public static class ComplexNestedObject1 { - + private double doubleValue; - + private List stringList; - - public ComplexNestedObject1() {} - + + public ComplexNestedObject1() { + } + public ComplexNestedObject1(int offInit) { this.doubleValue = 6293485.6723 + offInit; - + this.stringList = new ArrayList(); this.stringList.add("A" + offInit); this.stringList.add("somewhat" + offInit); @@ -197,7 +197,7 @@ public ComplexNestedObject1(int offInit) { this.stringList.add("of" + offInit); this.stringList.add("strings" + offInit); } - + @Override public boolean equals(Object obj) { if (obj.getClass() == ComplexNestedObject1.class) { @@ -208,18 +208,19 @@ public boolean equals(Object obj) { } } } - + public static class ComplexNestedObject2 { - + private long longValue; - + private Map theMap = new HashMap(); - - public ComplexNestedObject2() {} - + + public ComplexNestedObject2() { + } + public ComplexNestedObject2(Random rnd) { this.longValue = rnd.nextLong(); - + this.theMap.put(String.valueOf(rnd.nextLong()), new ComplexNestedObject1(rnd.nextInt())); this.theMap.put(String.valueOf(rnd.nextLong()), new ComplexNestedObject1(rnd.nextInt())); this.theMap.put(String.valueOf(rnd.nextLong()), new ComplexNestedObject1(rnd.nextInt())); @@ -227,7 +228,7 @@ public ComplexNestedObject2(Random rnd) { this.theMap.put(String.valueOf(rnd.nextLong()), new ComplexNestedObject1(rnd.nextInt())); this.theMap.put(String.valueOf(rnd.nextLong()), new ComplexNestedObject1(rnd.nextInt())); } - + @Override public boolean equals(Object obj) { if (obj.getClass() == ComplexNestedObject2.class) { @@ -238,21 +239,22 @@ public boolean equals(Object obj) { } } } - + public static class Book { private long bookId; private String title; private long authorId; - public Book() {} + public Book() { + } public Book(long bookId, String title, long authorId) { this.bookId = bookId; this.title = title; this.authorId = authorId; } - + @Override public boolean equals(Object obj) { if (obj.getClass() == Book.class) { @@ -270,14 +272,15 @@ public static class BookAuthor { private List bookTitles; private String authorName; - public BookAuthor() {} + public BookAuthor() { + } public BookAuthor(long authorId, List bookTitles, String authorName) { this.authorId = authorId; this.bookTitles = bookTitles; this.authorName = authorName; } - + @Override public boolean equals(Object obj) { if (obj.getClass() == BookAuthor.class) { diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleSerializerTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleSerializerTest.java index ef613f60449bc..0d9c0498bcfd5 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleSerializerTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/TupleSerializerTest.java @@ -27,11 +27,11 @@ import eu.stratosphere.api.java.tuple.Tuple5; import eu.stratosphere.api.java.typeutils.TupleTypeInfo; import eu.stratosphere.api.java.typeutils.TypeExtractor; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.Book; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.BookAuthor; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.ComplexNestedObject1; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.ComplexNestedObject2; -import eu.stratosphere.api.java.typeutils.runtime.GenericSerializerTest.SimpleTypes; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.Book; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.BookAuthor; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.ComplexNestedObject1; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.ComplexNestedObject2; +import eu.stratosphere.api.java.typeutils.runtime.GenericTypeSerializerTest.SimpleTypes; import eu.stratosphere.util.StringUtils; public class TupleSerializerTest { From be3a54b4e2cdee280f9945143337a132ffb72a09 Mon Sep 17 00:00:00 2001 From: Markus Holzemer Date: Mon, 12 May 2014 09:52:42 +0200 Subject: [PATCH 025/182] Created new SortedGrouping to move the group order from the Grouping --- .../api/java/operators/Grouping.java | 48 ++------ .../java/operators/ReduceGroupOperator.java | 7 +- .../api/java/operators/SortedGrouping.java | 110 ++++++++++++++++++ 3 files changed, 126 insertions(+), 39 deletions(-) create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java index 0d120e4809784..8c678e376bc15 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java @@ -14,8 +14,6 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.operators; -import java.util.Arrays; - import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.operators.Order; import eu.stratosphere.api.java.DataSet; @@ -38,12 +36,9 @@ */ public class Grouping { - private final DataSet dataSet; - - private final Keys keys; + protected final DataSet dataSet; - private int[] groupSortKeyPositions = null; - private Order[] groupSortOrders = null; + protected final Keys keys; public Grouping(DataSet set, Keys keys) { if (set == null || keys == null) { @@ -67,14 +62,6 @@ protected Keys getKeys() { return this.keys; } - protected int[] getGroupSortKeyPositions() { - return this.groupSortKeyPositions; - } - - protected Order[] getGroupSortOrders() { - return this.groupSortOrders; - } - // -------------------------------------------------------------------------------------------- // Operations / Transformations // -------------------------------------------------------------------------------------------- @@ -96,9 +83,6 @@ protected Order[] getGroupSortOrders() { * @see DataSet */ public AggregateOperator aggregate(Aggregations agg, int field) { - if(this.groupSortKeyPositions != null) { - throw new UnsupportedOperationException("Sorted groups not supported for Aggregation operation at the moment."); - } return new AggregateOperator(this, agg, field); } @@ -116,9 +100,6 @@ public AggregateOperator aggregate(Aggregations agg, int field) { * @see DataSet */ public ReduceOperator reduce(ReduceFunction reducer) { - if(this.groupSortKeyPositions != null) { - throw new UnsupportedOperationException("Sorted groups not supported for Aggregation operation at the moment."); - } return new ReduceOperator(this, reducer); } @@ -150,14 +131,16 @@ public ReduceGroupOperator reduceGroup(GroupReduceFunction reduc * * @param field The Tuple field on which the group is sorted. * @param order The Order in which the specified Tuple field is sorted. - * @return A Grouping with specified order of group element. + * @return A SortedGrouping with specified order of group element. * * @see Tuple * @see Order */ - public Grouping sortGroup(int field, Order order) { + public SortedGrouping sortGroup(int field, Order order) { int pos; + int[] groupSortKeyPositions; + Order[] groupSortOrders ; if (!dataSet.getType().isTupleType()) { throw new InvalidProgramException("Specifying order keys via field positions is only valid for tuple data types"); @@ -166,20 +149,13 @@ public Grouping sortGroup(int field, Order order) { throw new IllegalArgumentException("Order key out of tuple bounds."); } - if(this.groupSortKeyPositions == null) { - this.groupSortKeyPositions = new int[1]; - this.groupSortOrders = new Order[1]; - pos = 0; - } else { - int newLength = this.groupSortKeyPositions.length + 1; - this.groupSortKeyPositions = Arrays.copyOf(this.groupSortKeyPositions, newLength); - this.groupSortOrders = Arrays.copyOf(this.groupSortOrders, newLength); - pos = newLength - 1; - } + groupSortKeyPositions = new int[1]; + groupSortOrders = new Order[1]; + pos = 0; - this.groupSortKeyPositions[pos] = field; - this.groupSortOrders[pos] = order; - return this; + groupSortKeyPositions[pos] = field; + groupSortOrders[pos] = order; + return new SortedGrouping(this.dataSet, this.keys, groupSortKeyPositions, groupSortOrders); } // public > Grouping sortGroup(KeySelector keyExtractor, Order order) { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java index 680cc651faef1..392badc338092 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java @@ -148,10 +148,11 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { po.setDegreeOfParallelism(this.getParallelism()); // set group order - if(grouper.getGroupSortKeyPositions() != null) { + if(grouper instanceof SortedGrouping) { + SortedGrouping sortedGrouper = (SortedGrouping) grouper; - int[] sortKeyPositions = grouper.getGroupSortKeyPositions(); - Order[] sortOrders = grouper.getGroupSortOrders(); + int[] sortKeyPositions = sortedGrouper.getGroupSortKeyPositions(); + Order[] sortOrders = sortedGrouper.getGroupSortOrders(); Ordering o = new Ordering(); for(int i=0; i < sortKeyPositions.length; i++) { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java new file mode 100644 index 0000000000000..6190f8f4d33b9 --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java @@ -0,0 +1,110 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.operators; + +import java.util.Arrays; + +import eu.stratosphere.api.common.InvalidProgramException; +import eu.stratosphere.api.common.operators.Order; +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.functions.GroupReduceFunction; + +/** + * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.
            + * The following transformation can be applied on sorted groups: + *

              + *
            • {@link Grouping#reduce(ReduceFunction)},
            • + *
            + * + * @param The type of the elements of the sorted and grouped DataSet. + */ +public class SortedGrouping extends Grouping { + + private int[] groupSortKeyPositions; + private Order[] groupSortOrders ; + + public SortedGrouping(DataSet set, Keys keys, int[] groupSortKeyPositions, Order[] groupSortOrders) { + super(set, keys); + + if (groupSortKeyPositions == null || groupSortKeyPositions.length == 0 || groupSortOrders == null || groupSortOrders.length == 0) { + throw new InvalidProgramException("Key positions and sort orders must be specified in order to create a SortedGrouping."); + } + + this.groupSortKeyPositions = groupSortKeyPositions; + this.groupSortOrders = groupSortOrders; + } + + protected int[] getGroupSortKeyPositions() { + return this.groupSortKeyPositions; + } + + protected Order[] getGroupSortOrders() { + return this.groupSortOrders; + } + + /** + * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}.
            + * The transformation calls a {@link GroupReduceFunction} for each group of the DataSet. + * A GroupReduceFunction can iterate over all elements of a group and emit any + * number of output elements including none. + * + * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. + * @return A GroupReduceOperator that represents the reduced DataSet. + * + * @see GroupReduceFunction + * @see GroupReduceOperator + * @see DataSet + */ + public ReduceGroupOperator reduceGroup(GroupReduceFunction reducer) { + return new ReduceGroupOperator(this, reducer); + } + + // -------------------------------------------------------------------------------------------- + // Group Operations + // -------------------------------------------------------------------------------------------- + + /** + * Sorts {@link Tuple} elements within a group on the specified field in the specified {@link Order}.
            + * Note: Only groups of Tuple elements can be sorted.
            + * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. + * + * @param field The Tuple field on which the group is sorted. + * @param order The Order in which the specified Tuple field is sorted. + * @return A SortedGrouping with specified order of group element. + * + * @see Tuple + * @see Order + */ + public SortedGrouping sortGroup(int field, Order order) { + + int pos; + + if (!dataSet.getType().isTupleType()) { + throw new InvalidProgramException("Specifying order keys via field positions is only valid for tuple data types"); + } + if (field >= dataSet.getType().getArity()) { + throw new IllegalArgumentException("Order key out of tuple bounds."); + } + + int newLength = this.groupSortKeyPositions.length + 1; + this.groupSortKeyPositions = Arrays.copyOf(this.groupSortKeyPositions, newLength); + this.groupSortOrders = Arrays.copyOf(this.groupSortOrders, newLength); + pos = newLength - 1; + + this.groupSortKeyPositions[pos] = field; + this.groupSortOrders[pos] = order; + return this; + } +} From 1210db6fa2f3c069ce66b9b5118da4c95a8a2f18 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 14:06:20 +0200 Subject: [PATCH 026/182] Minor changes for SortedGrouping --- .../api/java/operators/Grouping.java | 24 +------------------ .../api/java/operators/SortedGrouping.java | 14 +++++++---- 2 files changed, 10 insertions(+), 28 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java index 8c678e376bc15..280f88b018e1a 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java @@ -137,29 +137,7 @@ public ReduceGroupOperator reduceGroup(GroupReduceFunction reduc * @see Order */ public SortedGrouping sortGroup(int field, Order order) { - - int pos; - int[] groupSortKeyPositions; - Order[] groupSortOrders ; - - if (!dataSet.getType().isTupleType()) { - throw new InvalidProgramException("Specifying order keys via field positions is only valid for tuple data types"); - } - if (field >= dataSet.getType().getArity()) { - throw new IllegalArgumentException("Order key out of tuple bounds."); - } - - groupSortKeyPositions = new int[1]; - groupSortOrders = new Order[1]; - pos = 0; - - groupSortKeyPositions[pos] = field; - groupSortOrders[pos] = order; - return new SortedGrouping(this.dataSet, this.keys, groupSortKeyPositions, groupSortOrders); + return new SortedGrouping(this.dataSet, this.keys, field, order); } - -// public > Grouping sortGroup(KeySelector keyExtractor, Order order) { -// throw new UnsupportedOperationException("Group sorting not supported for KeyExtractor functions."); -// } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java index 6190f8f4d33b9..3edb05ccff428 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java @@ -35,15 +35,19 @@ public class SortedGrouping extends Grouping { private int[] groupSortKeyPositions; private Order[] groupSortOrders ; - public SortedGrouping(DataSet set, Keys keys, int[] groupSortKeyPositions, Order[] groupSortOrders) { + public SortedGrouping(DataSet set, Keys keys, int field, Order order) { super(set, keys); - if (groupSortKeyPositions == null || groupSortKeyPositions.length == 0 || groupSortOrders == null || groupSortOrders.length == 0) { - throw new InvalidProgramException("Key positions and sort orders must be specified in order to create a SortedGrouping."); + if (!dataSet.getType().isTupleType()) { + throw new InvalidProgramException("Specifying order keys via field positions is only valid for tuple data types"); + } + if (field >= dataSet.getType().getArity()) { + throw new IllegalArgumentException("Order key out of tuple bounds."); } - this.groupSortKeyPositions = groupSortKeyPositions; - this.groupSortOrders = groupSortOrders; + this.groupSortKeyPositions = new int[]{field}; + this.groupSortOrders = new Order[]{order}; + } protected int[] getGroupSortKeyPositions() { From 032d5332a6c7fbf7de4f4aa179847d24a9aaa830 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 14:55:21 +0200 Subject: [PATCH 027/182] Updated JavaDocs of ReduceFunction --- .../eu/stratosphere/api/java/functions/ReduceFunction.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/ReduceFunction.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/ReduceFunction.java index e5de6a2980738..0ce21687f6fd5 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/ReduceFunction.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/ReduceFunction.java @@ -33,11 +33,8 @@ public abstract class ReduceFunction extends AbstractFunction implements Gene * The reduce function is consecutively applied to all values of a group until only a single value remains. * In functional programming, this is known as a fold-style aggregation. * - * Important: It is fine to return the second value object (value2) as result from this function. - * You must NOT return the first value object (value1) from this function. - * - * @param value1 The first value to combine. This object must NOT be returned as result. - * @param value2 The second value to combine. This object may be returned as result. + * @param value1 The first value to combine. + * @param value2 The second value to combine. * @return The combined value of both input values. * * @throws Exception From c951e4ec23c86157b4f65742df9189fb9d93fd14 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 14:56:16 +0200 Subject: [PATCH 028/182] Minor improvements in comparator tests --- .../typeutils/runtime/CopyableValueComparatorTest.java | 4 ++-- .../typeutils/runtime/GenericTypeComparatorTest.java | 2 ++ .../api/java/typeutils/runtime/ValueComparatorTest.java | 9 ++------- 3 files changed, 6 insertions(+), 9 deletions(-) diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java index 8a13571577d17..f4bb64da52113 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/CopyableValueComparatorTest.java @@ -34,12 +34,12 @@ public class CopyableValueComparatorTest extends ComparatorTestBase @Override protected TypeComparator createComparator(boolean ascending) { - return new CopyableValueComparator(ascending, StringValue.class); + return new CopyableValueComparator(ascending, StringValue.class); } @Override protected TypeSerializer createSerializer() { - return new CopyableValueSerializer(StringValue.class); + return new CopyableValueSerializer(StringValue.class); } @Override diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java index 2b8271cd282ec..aeed878e0d901 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/GenericTypeComparatorTest.java @@ -113,6 +113,7 @@ private class ComparatorTestInstance extends ComparatorTestBase { private final Class type; + @SuppressWarnings("unchecked") public ComparatorTestInstance(T[] testData) { if (testData == null || testData.length == 0) { throw new IllegalArgumentException(); @@ -123,6 +124,7 @@ public ComparatorTestInstance(T[] testData) { } @Override + @SuppressWarnings({ "unchecked", "rawtypes" }) protected TypeComparator createComparator(boolean ascending) { return new GenericTypeComparator(ascending, GenericTypeComparatorTest.createSerializer(this.type), this.type); } diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java index d00d4c1bae81f..34b586dcdb34c 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/ValueComparatorTest.java @@ -17,12 +17,7 @@ import eu.stratosphere.api.common.typeutils.ComparatorTestBase; import eu.stratosphere.api.common.typeutils.TypeComparator; import eu.stratosphere.api.common.typeutils.TypeSerializer; -import eu.stratosphere.api.common.typeutils.base.StringValueSerializer; import eu.stratosphere.types.StringValue; -import eu.stratosphere.types.Value; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; public class ValueComparatorTest extends ComparatorTestBase { @@ -39,12 +34,12 @@ public class ValueComparatorTest extends ComparatorTestBase { @Override protected TypeComparator createComparator(boolean ascending) { - return new ValueComparator(ascending, StringValue.class); + return new ValueComparator(ascending, StringValue.class); } @Override protected TypeSerializer createSerializer() { - return new ValueSerializer(StringValue.class); + return new ValueSerializer(StringValue.class); } @Override From 371dedfeeb4bce5b2a8a94ef9a70a20954b9bc07 Mon Sep 17 00:00:00 2001 From: twalthr Date: Wed, 7 May 2014 03:04:44 +0200 Subject: [PATCH 029/182] Introduction of a Writable type and tests --- .../api/java/ExecutionEnvironment.java | 2 +- .../stratosphere/api/java/operators/Keys.java | 2 +- .../api/java/typeutils/TypeExtractor.java | 27 ++- .../api/java/typeutils/TypeInformation.java | 29 +++- .../api/java/typeutils/ValueTypeInfo.java | 2 +- .../api/java/typeutils/WritableTypeInfo.java | 96 ++++++++++ .../typeutils/runtime/WritableComparator.java | 164 ++++++++++++++++++ .../typeutils/runtime/WritableSerializer.java | 102 +++++++++++ .../type/extractor/TypeExtractorTest.java | 55 +++++- .../java/typeutils/TypeInfoParserTest.java | 26 +++ .../runtime/StringArrayWritable.java | 79 +++++++++ .../runtime/WritableComparatorTest.java | 49 ++++++ .../runtime/WritableSerializerTest.java | 47 +++++ 13 files changed, 668 insertions(+), 12 deletions(-) create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparator.java create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializer.java create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/StringArrayWritable.java create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparatorTest.java create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializerTest.java diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java index e832f3ae4ff8e..b3569cbc8872d 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java @@ -277,7 +277,7 @@ public DataSource createInput(InputFormat inputFormat) { @SuppressWarnings("unchecked") TypeInformation producedType = (inputFormat instanceof ResultTypeQueryable) ? ((ResultTypeQueryable) inputFormat).getProducedType() : - TypeExtractor.extractInputFormatTypes(inputFormat); + TypeExtractor.getInputFormatTypes(inputFormat); return createInput(inputFormat, producedType); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java index c90b0797c76d3..e62a3e4d8bf8b 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java @@ -120,7 +120,7 @@ public static class SelectorFunctionKeys extends Keys { public SelectorFunctionKeys(KeySelector keyExtractor, TypeInformation type) { this.keyExtractor = keyExtractor; - this.keyType = TypeExtractor.getKeyExtractorType(keyExtractor, type); + this.keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, type); } public TypeInformation getKeyType() { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeExtractor.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeExtractor.java index 6f3011dd9947c..800f9f5876a21 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeExtractor.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeExtractor.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import org.apache.commons.lang3.Validate; +import org.apache.hadoop.io.Writable; import eu.stratosphere.api.common.io.InputFormat; import eu.stratosphere.api.java.functions.CoGroupFunction; @@ -74,13 +75,13 @@ public static TypeInformation getCrossReturnTypes(CrossFunc return createTypeInfo(CrossFunction.class, crossFunction.getClass(), 2, in1Type, in2Type); } - public static TypeInformation getKeyExtractorType(KeySelector selector, TypeInformation inType) { + public static TypeInformation getKeySelectorTypes(KeySelector selector, TypeInformation inType) { validateInputType(KeySelector.class, selector.getClass(), 0, inType); return createTypeInfo(KeySelector.class, selector.getClass(), 1, inType, null); } - public static TypeInformation extractInputFormatTypes(InputFormat format) { - throw new UnsupportedOperationException("not implemented yet"); + public static TypeInformation getInputFormatTypes(InputFormat inputFormat) { + return createTypeInfo(InputFormat.class, inputFormat.getClass(), 0, null, null); } // -------------------------------------------------------------------------------------------- @@ -203,6 +204,21 @@ else if (typeInfo.isTupleType()) { validateInfo(new ArrayList(typeHierarchy), subTypes[i], ((TupleTypeInfo) typeInfo).getTypeAt(i)); } } + // check for Writable + else if (typeInfo instanceof WritableTypeInfo) { + // check if writable at all + if (!(type instanceof Class && Writable.class.isAssignableFrom((Class) type))) { + throw new InvalidTypesException("Writable type expected."); + } + + // check writable type contents + Class clazz = null; + if (((WritableTypeInfo) typeInfo).getTypeClass() != (clazz = (Class) type)) { + throw new InvalidTypesException("Writable type '" + + ((WritableTypeInfo) typeInfo).getTypeClass().getCanonicalName() + "' expected but was '" + + clazz.getCanonicalName() + "'."); + } + } // check for basic array else if (typeInfo instanceof BasicArrayTypeInfo) { Type component = null; @@ -566,6 +582,11 @@ public static TypeInformation getForClass(Class clazz) { } } + // check for writable types + if(Writable.class.isAssignableFrom(clazz)) { + return (TypeInformation) WritableTypeInfo.getWritableTypeInfo((Class) clazz); + } + // check for basic types TypeInformation basicTypeInfo = BasicTypeInfo.getInfoFor(clazz); if (basicTypeInfo != null) { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeInformation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeInformation.java index dfd180ce558e6..1fa0c1aa559a0 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeInformation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/TypeInformation.java @@ -38,8 +38,10 @@ public abstract class TypeInformation { private static final String TUPLE_PACKAGE = "eu.stratosphere.api.java.tuple"; private static final String VALUE_PACKAGE = "eu.stratosphere.types"; + private static final String WRITABLE_PACKAGE = "org.apache.hadoop.io"; private static final Pattern tuplePattern = Pattern.compile("^((" + TUPLE_PACKAGE.replaceAll("\\.", "\\\\.") + "\\.)?Tuple[0-9]+)<"); + private static final Pattern writablePattern = Pattern.compile("^((" + WRITABLE_PACKAGE.replaceAll("\\.", "\\\\.") + "\\.)?Writable)<([^\\s,>]*)(,|>|$)"); private static final Pattern basicTypePattern = Pattern .compile("^((java\\.lang\\.)?(String|Integer|Byte|Short|Character|Double|Float|Long|Boolean))(,|>|$)"); private static final Pattern basicType2Pattern = Pattern.compile("^(int|byte|short|char|double|float|long|boolean)(,|>|$)"); @@ -61,13 +63,14 @@ public abstract class TypeInformation { * String[], etc. *
          • Tuple types such as Tuple1<TYPE0>, * Tuple2<TYPE0, TYPE1>, etc.
          • - *
          • Custom types such as org.my.CustomObject, - * org.my.CustomObject$StaticInnerClass, etc. - *
          • Custom type arrays such as org.my.CustomObject[], - * org.my.CustomObject$InnerClass[], etc. + *
          • Custom types such as org.my.CustomClass, + * org.my.CustomClass$StaticInnerClass, etc. + *
          • Custom type arrays such as org.my.CustomClass[], + * org.my.CustomClass$StaticInnerClass[], etc. *
          • Value types such as DoubleValue, * StringValue, IntegerValue, etc.
          • - *
          • Tuple arrays such as Tuple2[], etc.
          • + *
          • Tuple array types such as Tuple2[], etc.
          • + *
          • Writable types such as Writable<org.my.CustomWritable>
          • *
          * * Example: @@ -99,6 +102,8 @@ private static TypeInformation parse(StringBuilder sb) throws ClassNotFoundEx String infoString = sb.toString(); final Matcher tupleMatcher = tuplePattern.matcher(infoString); + final Matcher writableMatcher = writablePattern.matcher(infoString); + final Matcher basicTypeMatcher = basicTypePattern.matcher(infoString); final Matcher basicType2Matcher = basicType2Pattern.matcher(infoString); @@ -162,6 +167,20 @@ private static TypeInformation parse(StringBuilder sb) throws ClassNotFoundEx returnType = new TupleTypeInfo(clazz, types); } } + // writable types + else if (writableMatcher.find()) { + String className = writableMatcher.group(1); + String fullyQualifiedName = writableMatcher.group(3); + sb.delete(0, className.length() + 1 + fullyQualifiedName.length()); + + try { + Class clazz = Class.forName(fullyQualifiedName); + returnType = WritableTypeInfo.getWritableTypeInfo((Class) clazz); + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException("Class '" + fullyQualifiedName + + "' could not be found for use as writable type. Please note that inner classes must be declared static."); + } + } // basic types of classes else if (basicTypeMatcher.find()) { String className = basicTypeMatcher.group(1); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ValueTypeInfo.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ValueTypeInfo.java index 4aff6b3e0fe25..cdcc6e89dc9c9 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ValueTypeInfo.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ValueTypeInfo.java @@ -34,7 +34,7 @@ public ValueTypeInfo(Class type) { if (type == null) { throw new NullPointerException(); } - if (!Value.class.isAssignableFrom(type)) { + if (!Value.class.isAssignableFrom(type) && !type.equals(Value.class)) { throw new IllegalArgumentException("ValueTypeInfo can only be used for subclasses of " + Value.class.getName()); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java new file mode 100644 index 0000000000000..d13a31dac3ca2 --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java @@ -0,0 +1,96 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils; + +import org.apache.hadoop.io.Writable; + +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.TypeSerializer; +import eu.stratosphere.api.java.functions.InvalidTypesException; +import eu.stratosphere.api.java.typeutils.runtime.WritableComparator; +import eu.stratosphere.api.java.typeutils.runtime.WritableSerializer; + +public class WritableTypeInfo extends TypeInformation implements AtomicType { + + private final Class typeClass; + + public WritableTypeInfo(Class typeClass) { + if (typeClass == null) { + throw new NullPointerException(); + } + if (!Writable.class.isAssignableFrom(typeClass) || typeClass == Writable.class) { + throw new IllegalArgumentException("WritableTypeInfo can only be used for subclasses of " + Writable.class.getName()); + } + this.typeClass = typeClass; + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + @Override + public TypeComparator createComparator(boolean sortOrderAscending) { + if(Comparable.class.isAssignableFrom(typeClass)) { + return new WritableComparator(sortOrderAscending, typeClass); + } + else { + throw new UnsupportedOperationException("Writable does not implement Comparable interface."); + } + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public Class getTypeClass() { + return this.typeClass; + } + + @Override + public boolean isKeyType() { + return Comparable.class.isAssignableFrom(typeClass); + } + + @Override + public TypeSerializer createSerializer() { + return new WritableSerializer(typeClass); + } + + @Override + public String toString() { + return "WritableType<" + typeClass.getName() + ">"; + } + + // -------------------------------------------------------------------------------------------- + + static final TypeInformation getWritableTypeInfo(Class typeClass) { + if (Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class)) { + return new WritableTypeInfo(typeClass); + } + else { + throw new InvalidTypesException("The given class is no subclass of " + Writable.class.getName()); + } + } + +} diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparator.java new file mode 100644 index 0000000000000..0f8966c6c19b5 --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparator.java @@ -0,0 +1,164 @@ +/*********************************************************************************************************************** +* +* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) +* +* Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import java.io.IOException; + +import org.apache.hadoop.io.Writable; + +import com.esotericsoftware.kryo.Kryo; + +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; +import eu.stratosphere.core.memory.MemorySegment; +import eu.stratosphere.types.NormalizableKey; +import eu.stratosphere.util.InstantiationUtil; + +public class WritableComparator> extends TypeComparator { + + private static final long serialVersionUID = 1L; + + private Class type; + + private final boolean ascendingComparison; + + private transient T reference; + + private transient T tempReference; + + private transient Kryo kryo; + + public WritableComparator(boolean ascending, Class type) { + this.type = type; + this.ascendingComparison = ascending; + } + + @Override + public int hash(T record) { + return record.hashCode(); + } + + @Override + public void setReference(T toCompare) { + checkKryoInitialized(); + reference = this.kryo.copy(toCompare); + } + + @Override + public boolean equalToReference(T candidate) { + return candidate.equals(reference); + } + + @Override + public int compareToReference(TypeComparator referencedComparator) { + T otherRef = ((WritableComparator) referencedComparator).reference; + int comp = otherRef.compareTo(reference); + return ascendingComparison ? comp : -comp; + } + + @Override + public int compare(T first, T second) { + int comp = first.compareTo(second); + return ascendingComparison ? comp : -comp; + } + + @Override + public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException { + ensureReferenceInstantiated(); + ensureTempReferenceInstantiated(); + + reference.readFields(firstSource); + tempReference.readFields(secondSource); + + int comp = reference.compareTo(tempReference); + return ascendingComparison ? comp : -comp; + } + + @Override + public boolean supportsNormalizedKey() { + return NormalizableKey.class.isAssignableFrom(type); + } + + @Override + public int getNormalizeKeyLen() { + ensureReferenceInstantiated(); + + NormalizableKey key = (NormalizableKey) reference; + return key.getMaxNormalizedKeyLen(); + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return keyBytes < getNormalizeKeyLen(); + } + + @Override + public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { + NormalizableKey key = (NormalizableKey) record; + key.copyNormalizedKey(target, offset, numBytes); + } + + @Override + public boolean invertNormalizedKey() { + return !ascendingComparison; + } + + @Override + public TypeComparator duplicate() { + return new WritableComparator(ascendingComparison, type); + } + + // -------------------------------------------------------------------------------------------- + // unsupported normalization + // -------------------------------------------------------------------------------------------- + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException(); + } + + // -------------------------------------------------------------------------------------------- + + private final void checkKryoInitialized() { + if (this.kryo == null) { + this.kryo = new Kryo(); + this.kryo.setAsmEnabled(true); + this.kryo.register(type); + } + } + + private final void ensureReferenceInstantiated() { + if (reference == null) { + reference = InstantiationUtil.instantiate(type, Writable.class); + } + } + + private final void ensureTempReferenceInstantiated() { + if (tempReference == null) { + tempReference = InstantiationUtil.instantiate(type, Writable.class); + } + } +} diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializer.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializer.java new file mode 100644 index 0000000000000..61c7dae63c825 --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializer.java @@ -0,0 +1,102 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import java.io.IOException; + +import org.apache.hadoop.io.Writable; + +import com.esotericsoftware.kryo.Kryo; + +import eu.stratosphere.api.common.typeutils.TypeSerializer; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; +import eu.stratosphere.util.InstantiationUtil; + +public class WritableSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + private final Class typeClass; + + private transient Kryo kryo; + + private transient T copyInstance; + + public WritableSerializer(Class typeClass) { + this.typeClass = typeClass; + } + + @Override + public T createInstance() { + return InstantiationUtil.instantiate(typeClass); + } + + @Override + public T copy(T from, T reuse) { + checkKryoInitialized(); + reuse = this.kryo.copy(from); + return reuse; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(T record, DataOutputView target) throws IOException { + record.write(target); + } + + @Override + public T deserialize(T reuse, DataInputView source) throws IOException { + reuse.readFields(source); + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + ensureInstanceInstantiated(); + copyInstance.readFields(source); + copyInstance.write(target); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public boolean isStateful() { + return true; + } + + // -------------------------------------------------------------------------------------------- + + private final void ensureInstanceInstantiated() { + if (copyInstance == null) { + copyInstance = createInstance(); + } + } + + private final void checkKryoInitialized() { + if (this.kryo == null) { + this.kryo = new Kryo(); + this.kryo.setAsmEnabled(true); + this.kryo.register(typeClass); + } + } +} diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/TypeExtractorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/TypeExtractorTest.java index 4373ec2697be0..0ed4bcd9aea12 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/TypeExtractorTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/TypeExtractorTest.java @@ -14,8 +14,12 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.type.extractor; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.Iterator; +import org.apache.hadoop.io.Writable; import org.junit.Assert; import org.junit.Test; @@ -40,6 +44,7 @@ import eu.stratosphere.api.java.typeutils.TypeExtractor; import eu.stratosphere.api.java.typeutils.TypeInformation; import eu.stratosphere.api.java.typeutils.ValueTypeInfo; +import eu.stratosphere.api.java.typeutils.WritableTypeInfo; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.StringValue; @@ -75,6 +80,38 @@ public void reduce(Iterator values, Collector out) throws Exce // use getForObject() Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, TypeExtractor.getForObject(Boolean.valueOf(true))); } + + public static class MyWritable implements Writable { + + @Override + public void write(DataOutput out) throws IOException { + + } + + @Override + public void readFields(DataInput in) throws IOException { + } + + } + + @SuppressWarnings({ "unchecked", "rawtypes" }) + @Test + public void testWritableType() { + MapFunction function = new MapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public MyWritable map(MyWritable value) throws Exception { + return null; + } + + }; + + TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) new WritableTypeInfo(MyWritable.class)); + + Assert.assertTrue(ti instanceof WritableTypeInfo); + Assert.assertEquals(MyWritable.class, ((WritableTypeInfo) ti).getTypeClass()); + } @SuppressWarnings({ "unchecked", "rawtypes" }) @Test @@ -344,7 +381,7 @@ public StringValue getKey(StringValue value) { } }; - TypeInformation ti = TypeExtractor.getKeyExtractorType(function, (TypeInformation) TypeInformation.parse("StringValue")); + TypeInformation ti = TypeExtractor.getKeySelectorTypes(function, (TypeInformation) TypeInformation.parse("StringValue")); Assert.assertFalse(ti.isBasicType()); Assert.assertFalse(ti.isTupleType()); @@ -1249,5 +1286,21 @@ public String map(Tuple1[] value) throws Exception { } catch (InvalidTypesException e) { // right } + + MapFunction function4 = new MapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public String map(Writable value) throws Exception { + return null; + } + }; + + try { + TypeExtractor.getMapReturnTypes(function4, (TypeInformation) new WritableTypeInfo(MyWritable.class)); + Assert.fail("exception expected"); + } catch (InvalidTypesException e) { + // right + } } } diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/TypeInfoParserTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/TypeInfoParserTest.java index 866152efd9635..9b405862a464d 100644 --- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/TypeInfoParserTest.java +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/TypeInfoParserTest.java @@ -14,8 +14,13 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.typeutils; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + import junit.framework.Assert; +import org.apache.hadoop.io.Writable; import org.junit.Test; import eu.stratosphere.api.java.typeutils.BasicArrayTypeInfo; @@ -145,6 +150,27 @@ public void testCustomType() { Assert.assertEquals(Class.class, ((GenericTypeInfo) ti).getTypeClass()); } + public static class MyWritable implements Writable { + + @Override + public void write(DataOutput out) throws IOException { + + } + + @Override + public void readFields(DataInput in) throws IOException { + + } + + } + + @Test + public void testWritableType() { + TypeInformation ti = TypeInformation.parse("Writable"); + Assert.assertTrue(ti instanceof WritableTypeInfo); + Assert.assertEquals(MyWritable.class, ((WritableTypeInfo) ti).getTypeClass()); + } + @Test public void testObjectArrays() { TypeInformation ti = TypeInformation.parse("java.lang.Class[]"); diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/StringArrayWritable.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/StringArrayWritable.java new file mode 100644 index 0000000000000..c21515630f757 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/StringArrayWritable.java @@ -0,0 +1,79 @@ +/*********************************************************************************************************************** +* +* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) +* +* Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Writable; + +public class StringArrayWritable implements Writable, Comparable { + + private String[] array = new String[0]; + + public StringArrayWritable() { + super(); + } + + public StringArrayWritable(String[] array) { + this.array = array; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(this.array.length); + + for(String str : this.array) { + byte[] b = str.getBytes(); + out.writeInt(b.length); + out.write(b); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + this.array = new String[in.readInt()]; + + for(int i = 0; i < this.array.length; i++) { + byte[] b = new byte[in.readInt()]; + in.readFully(b); + this.array[i] = new String(b); + } + } + + @Override + public int compareTo(StringArrayWritable o) { + if(this.array.length != o.array.length) { + return this.array.length - o.array.length; + } + + for(int i = 0; i < this.array.length; i++) { + int comp = this.array[i].compareTo(o.array[i]); + if(comp != 0) { + return comp; + } + } + return 0; + } + + @Override + public boolean equals(Object obj) { + if(!(obj instanceof StringArrayWritable)) { + return false; + } + return this.compareTo((StringArrayWritable) obj) == 0; + } +} \ No newline at end of file diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparatorTest.java new file mode 100644 index 0000000000000..8f38b70ad9fb1 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableComparatorTest.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import eu.stratosphere.api.common.typeutils.ComparatorTestBase; +import eu.stratosphere.api.common.typeutils.TypeComparator; +import eu.stratosphere.api.common.typeutils.TypeSerializer; + +public class WritableComparatorTest extends ComparatorTestBase { + + StringArrayWritable[] data = new StringArrayWritable[]{ + new StringArrayWritable(new String[]{}), + new StringArrayWritable(new String[]{""}), + new StringArrayWritable(new String[]{"a","a"}), + new StringArrayWritable(new String[]{"a","b"}), + new StringArrayWritable(new String[]{"c","c"}), + new StringArrayWritable(new String[]{"d","f"}), + new StringArrayWritable(new String[]{"d","m"}), + new StringArrayWritable(new String[]{"z","x"}), + new StringArrayWritable(new String[]{"a","a", "a"}) + }; + + @Override + protected TypeComparator createComparator(boolean ascending) { + return new WritableComparator(ascending, StringArrayWritable.class); + } + + @Override + protected TypeSerializer createSerializer() { + return new WritableSerializer(StringArrayWritable.class); + } + + @Override + protected StringArrayWritable[] getSortedTestData() { + return data; + } +} diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializerTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializerTest.java new file mode 100644 index 0000000000000..a8e4b8d226659 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/WritableSerializerTest.java @@ -0,0 +1,47 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.typeutils.runtime; + +import org.junit.Test; + +import eu.stratosphere.api.common.typeutils.SerializerTestInstance; +import eu.stratosphere.api.java.typeutils.TypeExtractor; +import eu.stratosphere.api.java.typeutils.WritableTypeInfo; + +public class WritableSerializerTest { + + @Test + public void testStringArrayWritable() { + StringArrayWritable[] data = new StringArrayWritable[]{ + new StringArrayWritable(new String[]{}), + new StringArrayWritable(new String[]{""}), + new StringArrayWritable(new String[]{"a","a"}), + new StringArrayWritable(new String[]{"a","b"}), + new StringArrayWritable(new String[]{"c","c"}), + new StringArrayWritable(new String[]{"d","f"}), + new StringArrayWritable(new String[]{"d","m"}), + new StringArrayWritable(new String[]{"z","x"}), + new StringArrayWritable(new String[]{"a","a", "a"}) + }; + + WritableTypeInfo writableTypeInfo = (WritableTypeInfo) TypeExtractor.getForObject(data[0]); + WritableSerializer writableSerializer = (WritableSerializer) writableTypeInfo.createSerializer(); + + SerializerTestInstance testInstance = new SerializerTestInstance(writableSerializer,writableTypeInfo.getTypeClass(), -1, data); + + testInstance.testAll(); + } + +} From 957f01c199c2122fab49364529979fd4b2ccd532 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 17:52:14 +0200 Subject: [PATCH 030/182] Improved exception message in WritableTypeInfo. --- .../eu/stratosphere/api/java/typeutils/WritableTypeInfo.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java index d13a31dac3ca2..56fc7e1608322 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/WritableTypeInfo.java @@ -43,7 +43,8 @@ public TypeComparator createComparator(boolean sortOrderAscending) { return new WritableComparator(sortOrderAscending, typeClass); } else { - throw new UnsupportedOperationException("Writable does not implement Comparable interface."); + throw new UnsupportedOperationException("Cannot create Comparator for "+typeClass.getCanonicalName()+". " + + "Class does not implement Comparable interface."); } } From efa6e06d2405a588fbe0805f28960c004d80ae28 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Thu, 15 May 2014 17:19:26 +0200 Subject: [PATCH 031/182] Fix combiner for group reduce. --- .../operators/AllGroupReduceProperties.java | 2 +- ...AllGroupWithPartialPreGroupProperties.java | 6 +- .../compiler/postpass/JavaApiPostPass.java | 22 ++- .../runtime/task/AllGroupReduceDriver.java | 27 +++- .../pact/runtime/task/DriverStrategy.java | 2 + .../javaApiOperators/GroupReduceITCase.java | 153 ++++++++++++------ .../test/javaApiOperators/MapITCase.java | 4 +- 7 files changed, 146 insertions(+), 70 deletions(-) diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupReduceProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupReduceProperties.java index b2f5384329069..ee03ce01a7e35 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupReduceProperties.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupReduceProperties.java @@ -34,7 +34,7 @@ public DriverStrategy getStrategy() { @Override public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", in, DriverStrategy.ALL_GROUP_REDUCE); + return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")", in, DriverStrategy.ALL_GROUP_REDUCE); } @Override diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java index 201d8f7446759..d387c69a3f9d3 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/operators/AllGroupWithPartialPreGroupProperties.java @@ -39,7 +39,7 @@ public DriverStrategy getStrategy() { public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { if (in.getShipStrategy() == ShipStrategyType.FORWARD) { // locally connected, directly instantiate - return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", in, DriverStrategy.ALL_GROUP_REDUCE); + return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")", in, DriverStrategy.ALL_GROUP_REDUCE); } else { // non forward case.plug in a combiner Channel toCombiner = new Channel(in.getSource()); @@ -50,14 +50,14 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism()); combinerNode.setSubtasksPerInstance(in.getSource().getSubtasksPerInstance()); - SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_REDUCE); + SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_COMBINE); combiner.setCosts(new Costs(0, 0)); combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties()); Channel toReducer = new Channel(combiner); toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(), in.getShipStrategySortOrder()); toReducer.setLocalStrategy(in.getLocalStrategy(), in.getLocalStrategyKeys(), in.getLocalStrategySortOrder()); - return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", toReducer, DriverStrategy.ALL_GROUP_REDUCE); + return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")", toReducer, DriverStrategy.ALL_GROUP_REDUCE); } } diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/postpass/JavaApiPostPass.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/postpass/JavaApiPostPass.java index 2723b9a151066..795213d6dbc5d 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/postpass/JavaApiPostPass.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/postpass/JavaApiPostPass.java @@ -233,7 +233,8 @@ private void traverseChannel(Channel channel) { TypeInformation type = null; - if(javaOp instanceof PlanGroupReduceOperator && source.getDriverStrategy().equals(DriverStrategy.SORTED_GROUP_COMBINE)) { + if (javaOp instanceof PlanGroupReduceOperator && + (source.getDriverStrategy() == DriverStrategy.SORTED_GROUP_COMBINE || source.getDriverStrategy() == DriverStrategy.ALL_GROUP_COMBINE) ) { PlanGroupReduceOperator groupNode = (PlanGroupReduceOperator) javaOp; type = groupNode.getInputType(); } @@ -244,28 +245,33 @@ else if(javaOp instanceof PlanUnwrappingReduceGroupOperator && source.getDriverS else if (javaOp instanceof JavaPlanNode) { JavaPlanNode javaNode = (JavaPlanNode) javaOp; type = javaNode.getReturnType(); - } else if (javaOp instanceof BulkIteration.PartialSolutionPlaceHolder) { + } + else if (javaOp instanceof BulkIteration.PartialSolutionPlaceHolder) { BulkIteration.PartialSolutionPlaceHolder partialSolutionPlaceHolder = (BulkIteration.PartialSolutionPlaceHolder) javaOp; type = ((PlanBulkIterationOperator)partialSolutionPlaceHolder.getContainingBulkIteration()).getReturnType(); - } else if (javaOp instanceof DeltaIteration.SolutionSetPlaceHolder) { + } + else if (javaOp instanceof DeltaIteration.SolutionSetPlaceHolder) { DeltaIteration.SolutionSetPlaceHolder solutionSetPlaceHolder = (DeltaIteration.SolutionSetPlaceHolder) javaOp; type = ((PlanDeltaIterationOperator) solutionSetPlaceHolder.getContainingWorksetIteration()).getReturnType(); - } else if (javaOp instanceof DeltaIteration.WorksetPlaceHolder) { + } + else if (javaOp instanceof DeltaIteration.WorksetPlaceHolder) { DeltaIteration.WorksetPlaceHolder worksetPlaceHolder = (DeltaIteration.WorksetPlaceHolder) javaOp; type = ((PlanDeltaIterationOperator) worksetPlaceHolder.getContainingWorksetIteration()).getReturnType(); - } else if (javaOp instanceof NoOpUnaryUdfOp) { + } + else if (javaOp instanceof NoOpUnaryUdfOp) { NoOpUnaryUdfOp op = (NoOpUnaryUdfOp) javaOp; - if(op.getInput() instanceof JavaPlanNode) { + if (op.getInput() instanceof JavaPlanNode) { JavaPlanNode javaNode = (JavaPlanNode) op.getInput(); type = javaNode.getReturnType(); } - }else if(javaOp instanceof Union){ + } + else if(javaOp instanceof Union){ // Union Operator op = channel.getSource().getInputs().next().getSource().getPactContract(); - if(op instanceof JavaPlanNode){ + if (op instanceof JavaPlanNode){ JavaPlanNode javaNode = (JavaPlanNode) op; type = javaNode.getReturnType(); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AllGroupReduceDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AllGroupReduceDriver.java index 63186517bc996..2b3d0694599eb 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AllGroupReduceDriver.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AllGroupReduceDriver.java @@ -16,6 +16,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.api.common.functions.GenericCombine; import eu.stratosphere.api.common.functions.GenericGroupReduce; import eu.stratosphere.api.common.typeutils.TypeSerializer; import eu.stratosphere.pact.runtime.task.util.TaskConfig; @@ -42,6 +43,8 @@ public class AllGroupReduceDriver implements PactDriver input; private TypeSerializer serializer; + + private DriverStrategy strategy; // ------------------------------------------------------------------------ @@ -72,7 +75,14 @@ public boolean requiresComparatorOnInput() { @Override public void prepare() throws Exception { final TaskConfig config = this.taskContext.getTaskConfig(); - if (config.getDriverStrategy() != DriverStrategy.ALL_GROUP_REDUCE) { + this.strategy = config.getDriverStrategy(); + + if (strategy == DriverStrategy.ALL_GROUP_COMBINE) { + if (!(this.taskContext.getStub() instanceof GenericCombine)) { + throw new Exception("Using combiner on a UDF that does not implement the combiner interface " + GenericCombine.class.getName()); + } + } + else if (strategy != DriverStrategy.ALL_GROUP_REDUCE) { throw new Exception("Unrecognized driver strategy for AllGroupReduce driver: " + config.getDriverStrategy().name()); } this.serializer = this.taskContext.getInputSerializer(0).getSerializer(); @@ -85,13 +95,22 @@ public void run() throws Exception { LOG.debug(this.taskContext.formatLogString("AllGroupReduce preprocessing done. Running Reducer code.")); } - final GenericGroupReduce stub = this.taskContext.getStub(); - final Collector output = this.taskContext.getOutputCollector(); final MutableToRegularIteratorWrapper inIter = new MutableToRegularIteratorWrapper(this.input, this.serializer); // single UDF call with the single group if (inIter.hasNext()) { - stub.reduce(inIter, output); + if (strategy == DriverStrategy.ALL_GROUP_REDUCE) { + final GenericGroupReduce reducer = this.taskContext.getStub(); + final Collector output = this.taskContext.getOutputCollector(); + reducer.reduce(inIter, output); + } + else { + @SuppressWarnings("unchecked") + final GenericCombine combiner = (GenericCombine) this.taskContext.getStub(); + @SuppressWarnings("unchecked") + final Collector output = (Collector) this.taskContext.getOutputCollector(); + combiner.combine(inIter, output); + } } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DriverStrategy.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DriverStrategy.java index d7cc0380491c1..f96a55360ad65 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DriverStrategy.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DriverStrategy.java @@ -44,6 +44,8 @@ public enum DriverStrategy { ALL_REDUCE(AllReduceDriver.class, null, PIPELINED, false), // group everything together into one group and apply the GroupReduce function ALL_GROUP_REDUCE(AllGroupReduceDriver.class, null, PIPELINED, false), + // group everything together into one group and apply the GroupReduce's combine function + ALL_GROUP_COMBINE(AllGroupReduceDriver.class, null, PIPELINED, false), // grouping the inputs and apply the Reduce Function SORTED_REDUCE(ReduceDriver.class, null, PIPELINED, true), diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/GroupReduceITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/GroupReduceITCase.java index de20a9706146f..3757fcdfdcd09 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/GroupReduceITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/GroupReduceITCase.java @@ -29,19 +29,22 @@ import eu.stratosphere.api.java.ExecutionEnvironment; import eu.stratosphere.api.java.functions.GroupReduceFunction; import eu.stratosphere.api.java.functions.KeySelector; +import eu.stratosphere.api.java.functions.MapFunction; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.tuple.Tuple3; import eu.stratosphere.api.java.tuple.Tuple5; +import eu.stratosphere.compiler.PactCompiler; import eu.stratosphere.configuration.Configuration; import eu.stratosphere.test.javaApiOperators.util.CollectionDataSets; import eu.stratosphere.test.javaApiOperators.util.CollectionDataSets.CustomType; import eu.stratosphere.test.util.JavaProgramTestBase; import eu.stratosphere.util.Collector; +@SuppressWarnings("serial") @RunWith(Parameterized.class) public class GroupReduceITCase extends JavaProgramTestBase { - private static int NUM_PROGRAMS = 11; + private static int NUM_PROGRAMS = 13; private int curProgId = config.getInteger("ProgramId", -1); private String resultPath; @@ -352,50 +355,55 @@ public Integer getKey(CustomType in) { "65,test5\n" + "111,test6\n"; } - // TODO: all-groupreduce with combine -// case 12: { -// -// /* -// * check correctness of all-groupreduce for tuples with combine -// */ -// -// final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -// -// DataSet> ds = CollectionDataSets.get3TupleDataSet(env); -// DataSet> reduceDs = ds.reduceGroup(new Tuple3GroupReduceWithCombine()); -// -// reduceDs.writeAsCsv(resultPath); -// env.execute(); -// -// // return expected result -// return "231,91,Hello World\n"; -// } - // TODO: descending sort not working -// case 10: { -// -// /* -// * check correctness of groupReduce on tuples with key field selector and group sorting -// */ -// -// final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -// env.setDegreeOfParallelism(1); -// -// DataSet> ds = CollectionDataSets.get3TupleDataSet(env); -// DataSet> reduceDs = ds. -// groupBy(1).sortGroup(2,Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce()); -// -// reduceDs.writeAsCsv(resultPath); -// env.execute(); -// -// // return expected result -// return "1,1,Hi\n" + -// "5,2,Hello world-Hello\n" + -// "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + -// "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + -// "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + -// "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; -// -// } + // all-groupreduce with combine + case 12: { + + /* + * check correctness of all-groupreduce for tuples with combine + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env) + .map(new IdentityMapper>()).setParallelism(4); + + Configuration cfg = new Configuration(); + cfg.setString(PactCompiler.HINT_SHIP_STRATEGY, PactCompiler.HINT_SHIP_STRATEGY_REPARTITION); + DataSet> reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine()) + .withParameters(cfg); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "322,testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n"; + } + // descending sort not working + case 13: { + + /* + * check correctness of groupReduce on tuples with key field selector and group sorting + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setDegreeOfParallelism(1); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + DataSet> reduceDs = ds. + groupBy(1).sortGroup(2,Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce()); + + reduceDs.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "1,1,Hi\n" + + "5,2,Hello world-Hello\n" + + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" + + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" + + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" + + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; + + } default: throw new IllegalArgumentException("Invalid program id"); } @@ -605,34 +613,70 @@ public void reduce(Iterator> values, @eu.stratosphere.api.java.functions.GroupReduceFunction.Combinable public static class Tuple3GroupReduceWithCombine extends GroupReduceFunction, Tuple2> { private static final long serialVersionUID = 1L; - + @Override public void combine(Iterator> values, Collector> out) throws Exception { - + Tuple3 o = new Tuple3(0, 0l, ""); - + while(values.hasNext()) { Tuple3 t = values.next(); o.f0 += t.f0; o.f1 = t.f1; o.f2 = "test"+o.f1; } - + out.collect(o); } @Override public void reduce(Iterator> values, Collector> out) throws Exception { - + int i = 0; String s = ""; - + while(values.hasNext()) { Tuple3 t = values.next(); i += t.f0; s = t.f2; } + + out.collect(new Tuple2(i, s)); + + } + } + + @eu.stratosphere.api.java.functions.GroupReduceFunction.Combinable + public static class Tuple3AllGroupReduceWithCombine extends GroupReduceFunction, Tuple2> { + private static final long serialVersionUID = 1L; + + @Override + public void combine(Iterator> values, Collector> out) { + + Tuple3 o = new Tuple3(0, 0l, ""); + + while(values.hasNext()) { + Tuple3 t = values.next(); + o.f0 += t.f0; + o.f1 += t.f1; + o.f2 += "test"; + } + + out.collect(o); + } + + @Override + public void reduce(Iterator> values, Collector> out) { + + int i = 0; + String s = ""; + + while(values.hasNext()) { + Tuple3 t = values.next(); + i += t.f0 + t.f1; + s += t.f2; + } out.collect(new Tuple2(i, s)); @@ -675,4 +719,11 @@ public void reduce(Iterator values, } } + + public static final class IdentityMapper extends MapFunction { + + @Override + public T map(T value) { return value; } + + } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java index 01531f25e7131..07d1b07143a03 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/MapITCase.java @@ -456,7 +456,6 @@ public Tuple3 map(Tuple3 value) DataSet> bcMapDs = ds. map(new MapFunction, Tuple3>() { private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); @Override public void open(Configuration config) { @@ -465,8 +464,7 @@ public void open(Configuration config) { } @Override - public Tuple3 map(Tuple3 value) - throws Exception { + public Tuple3 map(Tuple3 value) { return value; } }).withParameters(conf); From 1d6b3cea6a69b19d7e12c2c96e71d65813607981 Mon Sep 17 00:00:00 2001 From: sebastian kunert Date: Wed, 7 May 2014 00:00:26 +0200 Subject: [PATCH 032/182] implemented infrastructure for string style annotations --- .../java/functions/FunctionAnnotation.java | 285 ++++-------------- .../translation/PlanFlatMapOperator.java | 12 +- 2 files changed, 71 insertions(+), 226 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java index ca3664862524a..a0883b1c28d93 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java @@ -13,10 +13,13 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.functions; +import java.lang.annotation.Annotation; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import java.util.HashSet; +import java.util.Set; import com.google.common.primitives.Ints; @@ -81,35 +84,9 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFields { - int[] value() default {}; - int[] outTuplePos() default {}; - String[] inCustomPos() default {}; - String[] outCustomPos() default {}; + String[] value(); } - /** - * Specifies that all fields of an input tuple or custom object that are unchanged in the output of - * a {@link MapFunction}, or {@link ReduceFunction}). - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * - * This annotation is mutually exclusive with the {@link ConstantFieldsExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsExcept} annotation is not set, it is - * assumed that no field is constant. - */ - @Target(ElementType.TYPE) - @Retention(RetentionPolicy.RUNTIME) - public @interface AllFieldsConstants {} - /** * Specifies the fields of an input tuple or custom object of the first input that are unchanged in * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) @@ -139,10 +116,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsFirst { - int[] value() default {}; - int[] outTuplePos() default {}; - String[] inCustomPos() default {}; - String[] outCustomPos() default {}; + String[] value(); } /** @@ -173,10 +147,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsSecond { - int[] value() default {}; - int[] outTuplePos() default {}; - String[] outCustomPos() default {}; - String[] inCustomPos() default {}; + String[] value(); } /** @@ -208,8 +179,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsExcept { - int[] value() default {}; - String[] inCustomPos() default {}; + String[] value(); } /** @@ -241,8 +211,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsFirstExcept { - int[] value() default {}; - String[] inCustomPos() default {}; + String[] value(); } @@ -275,8 +244,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsSecondExcept { - int[] value() default {}; - String[] inCustomPos() default {}; + String[] value(); } /** @@ -287,8 +255,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFields { - int[] value() default {}; - String[] inCustomPos() default {}; + String[] value(); } /** @@ -299,8 +266,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsSecond { - int[] value() default {}; - String[] inCustomPos() default {}; + String[] value(); } /** @@ -311,8 +277,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsFirst { - int[] value() default {}; - String[] inCustomPos() default {}; + String[] value(); } /** * Private constructor to prevent instantiation. This class is intended only as a container. @@ -322,77 +287,6 @@ private FunctionAnnotation() {} // -------------------------------------------------------------------------------------------- // Function Annotation Handling // -------------------------------------------------------------------------------------------- - private static boolean checkValidity(ConstantFields constantSet) { - int counter = 0; - if (constantSet.value().length > 0) { - counter++; - }; - - if (constantSet.outTuplePos().length > 0) { - counter++; - }; - - if (constantSet.outCustomPos().length > 0) { - counter++; - }; - - if (constantSet.inCustomPos().length > 0) { - counter++; - }; - - if (counter > 2) { - return false; - } - return true; - } - - private static boolean checkValidity(ConstantFieldsFirst constantSet) { - int counter = 0; - if (constantSet.value().length > 0) { - counter++; - }; - - if (constantSet.outTuplePos().length > 0) { - counter++; - }; - - if (constantSet.outCustomPos().length > 0) { - counter++; - }; - - if (constantSet.inCustomPos().length > 0) { - counter++; - }; - - if (counter > 2) { - return false; - } - return true; - } - - private static boolean checkValidity(ConstantFieldsSecond constantSet) { - int counter = 0; - if (constantSet.value().length > 0) { - counter++; - }; - - if (constantSet.outTuplePos().length > 0) { - counter++; - }; - - if (constantSet.outCustomPos().length > 0) { - counter++; - }; - - if (constantSet.inCustomPos().length > 0) { - counter++; - }; - - if (counter > 2) { - return false; - } - return true; - } /** * Reads the annotations of a user defined function with one input and returns semantic properties according to the constant fields annotated. @@ -402,68 +296,38 @@ private static boolean checkValidity(ConstantFieldsSecond constantSet) { * @return The DualInputSemanticProperties containing the constant fields. */ - public static SingleInputSemanticProperties readSingleConstantAnnotations(UserCodeWrapper udf, TypeInformation input, TypeInformation output) { - if (!input.isTupleType() || !output.isTupleType()) { - return null; - } - - - AllFieldsConstants allConstants = udf.getUserCodeAnnotation(AllFieldsConstants.class); + public static Set readSingleConstantAnnotations(UserCodeWrapper udf) { ConstantFields constantSet = udf.getUserCodeAnnotation(ConstantFields.class); ConstantFieldsExcept notConstantSet = udf.getUserCodeAnnotation(ConstantFieldsExcept.class); ReadFields readfieldSet = udf.getUserCodeAnnotation(ReadFields.class); - - - int inputArity = input.getArity(); - int outputArity = output.getArity(); - if (notConstantSet != null && (constantSet != null || allConstants != null)) { - throw new RuntimeException("Either ConstantFields or ConstantFieldsExcept can be specified, not both."); - } + Set result = null; - if (constantSet != null && !checkValidity(constantSet)) { - throw new RuntimeException("Only two parameters of the annotation should be used at once."); - } - - SingleInputSemanticProperties semanticProperties = new SingleInputSemanticProperties(); - - if (readfieldSet != null && readfieldSet.value().length > 0) { - semanticProperties.setReadFields(new FieldSet(readfieldSet.value())); + if (notConstantSet != null && constantSet != null) { + throw new RuntimeException("Either ConstantFields or ConstantFieldsExcept can be specified, not both."); } - // extract notConstantSet from annotation - if (notConstantSet != null && notConstantSet.value().length > 0) { - for (int i = 0; i < inputArity && i < outputArity; i++) { - if (!Ints.contains(notConstantSet.value(), i)) { - semanticProperties.addForwardedField(i, i); - }; + if (notConstantSet != null) { + if (result == null) { + result = new HashSet(); } + result.add(notConstantSet); } - - if (allConstants != null) { - for (int i = 0; i < inputArity && i < outputArity; i++) { - semanticProperties.addForwardedField(i, i); + if (constantSet != null) { + if (result == null) { + result = new HashSet(); } + result.add(constantSet); } - - // extract constantSet from annotation - if (constantSet != null) { - if (constantSet.outTuplePos().length == 0 && constantSet.value().length > 0) { - for (int value: constantSet.value()) { - semanticProperties.addForwardedField(value,value); - } - } else if (constantSet.value().length == constantSet.outTuplePos().length && constantSet.value().length > 0) { - for (int i = 0; i < constantSet.value().length; i++) { - semanticProperties.addForwardedField(constantSet.value()[i], constantSet.outTuplePos()[i]); - } - } else { - throw new RuntimeException("Field 'from' and 'to' of the annotation should have the same length."); + if (readfieldSet != null) { + if (result == null) { + result = new HashSet(); } + result.add(readfieldSet); } - return semanticProperties; - + return result; } // -------------------------------------------------------------------------------------------- @@ -476,15 +340,8 @@ public static SingleInputSemanticProperties readSingleConstantAnnotations(UserCo * @return The DualInputSemanticProperties containing the constant fields. */ - public static DualInputSemanticProperties readDualConstantAnnotations(UserCodeWrapper udf, TypeInformation input1, TypeInformation input2, TypeInformation output) { - if (!input1.isTupleType() || !input2.isTupleType() || !output.isTupleType()) { - return null; - } - - int input1Arity = input1.getArity(); - int input2Arity = input2.getArity(); - int outputArity = output.getArity(); - + public static Set readDualConstantAnnotations(UserCodeWrapper udf) { + // get readSet annotation from stub ConstantFieldsFirst constantSet1 = udf.getUserCodeAnnotation(ConstantFieldsFirst.class); ConstantFieldsSecond constantSet2= udf.getUserCodeAnnotation(ConstantFieldsSecond.class); @@ -504,67 +361,49 @@ public static DualInputSemanticProperties readDualConstantAnnotations(UserCodeWr throw new RuntimeException("Either ConstantFieldsSecond or ConstantFieldsSecondExcept can be specified, not both."); } - if (constantSet1 != null && constantSet2 != null && (!checkValidity(constantSet1) || !checkValidity(constantSet2))) { - throw new RuntimeException("Only two parameters of the annotation should be used at once."); - } - - DualInputSemanticProperties semanticProperties = new DualInputSemanticProperties(); + Set result = null; - if (readfieldSet1 != null && readfieldSet2.value().length > 0) { - semanticProperties.setReadFields1(new FieldSet(readfieldSet1.value())); + if (notConstantSet2 != null) { + if (result == null) { + result = new HashSet(); + } + result.add(notConstantSet2); } - - if (readfieldSet2 != null && readfieldSet2.value().length > 0) { - semanticProperties.setReadFields2(new FieldSet(readfieldSet2.value())); + if (constantSet2 != null) { + if (result == null) { + result = new HashSet(); + } + result.add(constantSet2); } - // extract readSets from annotations - if(notConstantSet1 != null && notConstantSet1.value().length > 0) { - for (int i = 0; i < input1Arity && i < outputArity; i++) { - if (!Ints.contains(notConstantSet1.value(), i)) { - semanticProperties.addForwardedField1(i, i);; - }; + if (readfieldSet2 != null) { + if (result == null) { + result = new HashSet(); } + result.add(readfieldSet2); } - - if(notConstantSet2 != null && notConstantSet2.value().length > 0) { - for (int i = 0; i < input2Arity && i < outputArity; i++) { - if (!Ints.contains(notConstantSet2.value(), i)) { - semanticProperties.addForwardedField2(i, i);; - }; - } + + if (notConstantSet1 != null) { + if (result == null) { + result = new HashSet(); + } + result.add(notConstantSet1); } - - // extract readSets from annotations if (constantSet1 != null) { - if (constantSet1.outTuplePos().length == 0 && constantSet1.value().length > 0) { - for (int value: constantSet1.value()) { - semanticProperties.addForwardedField1(value,value); - } - } else if (constantSet1.value().length == constantSet1.outTuplePos().length && constantSet1.value().length > 0) { - for (int i = 0; i < constantSet1.value().length; i++) { - semanticProperties.addForwardedField1(constantSet1.value()[i], constantSet1.outTuplePos()[i]); - } - } else { - throw new RuntimeException("Field 'from' and 'to' of the annotation should have the same length."); + if (result == null) { + result = new HashSet(); } + result.add(constantSet1); } - - if (constantSet2 != null) { - if (constantSet2.outTuplePos().length == 0 && constantSet1.value().length > 0) { - for (int value: constantSet2.value()) { - semanticProperties.addForwardedField1(value,value); - } - } else if (constantSet2.value().length == constantSet2.outTuplePos().length && constantSet2.value().length > 0) { - for (int i = 0; i < constantSet2.value().length; i++) { - semanticProperties.addForwardedField2(constantSet2.value()[i], constantSet2.outTuplePos()[i]); - } - } else { - throw new RuntimeException("Field 'from' and 'to' of the ConstantFields annotation should have the same length."); + + if (readfieldSet1 != null) { + if (result == null) { + result = new HashSet(); } + result.add(readfieldSet1); } - - return semanticProperties; + + return result; } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java index 71a3f51620c51..147fbb3c7227b 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java @@ -14,14 +14,17 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.operators.translation; +import java.lang.annotation.Annotation; +import java.util.Set; + import eu.stratosphere.api.common.functions.GenericFlatMap; import eu.stratosphere.api.common.operators.base.FlatMapOperatorBase; import eu.stratosphere.api.java.functions.FlatMapFunction; +import eu.stratosphere.api.java.functions.FunctionAnnotation; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFields; import eu.stratosphere.api.java.typeutils.TypeInformation; -/** - * - */ + public class PlanFlatMapOperator extends FlatMapOperatorBase> implements UnaryJavaPlanNode { @@ -34,6 +37,9 @@ public PlanFlatMapOperator(FlatMapFunction udf, String name, TypeInformati super(udf, name); this.inType = inType; this.outType = outType; + + Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); + System.out.println(annotations); } @Override From 06a42052d2716787ee94389ea8f93f222cd259bb Mon Sep 17 00:00:00 2001 From: sebastian kunert Date: Wed, 7 May 2014 00:14:09 +0200 Subject: [PATCH 033/182] added semanticpropertyutilities --- .../api/java/functions/SemanticPropUtil.java | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java new file mode 100644 index 0000000000000..04477ff2f339d --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java @@ -0,0 +1,52 @@ +package eu.stratosphere.api.java.functions; + +import java.lang.annotation.Annotation; +import java.util.Iterator; +import java.util.Set; + +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFields; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsExcept; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ReadFields; +import eu.stratosphere.api.java.typeutils.TypeInformation; + +public class SemanticPropUtil { + public SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { + Iterator it = set.iterator(); + SingleInputSemanticProperties result = null; + + while (it.hasNext()) { + if (result == null) { + result = new SingleInputSemanticProperties(); + } + + Annotation ann = it.next(); + + if (ann instanceof ConstantFields) { + ConstantFields cf = (ConstantFields) ann; + } else if (ann instanceof ConstantFieldsExcept) { + ConstantFieldsExcept cfe = (ConstantFieldsExcept) ann; + } else if (ann instanceof ReadFields) { + ReadFields rf = (ReadFields) ann; + } + } + return null; + } + + private void parseConstantFields(ConstantFields cf, SingleInputSemanticProperties sm) { + + } + + private void parseConstantFieldsExcept(ConstantFieldsExcept cfe, SingleInputSemanticProperties sm) { + + } + + private void parseReadFields(ReadFields rf, SingleInputSemanticProperties sm) { + + } + + public DualInputSemanticProperties getSemanticPropsDua(Set set, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { + return null; + } +} From 3985eebb764eabc02ad6bf181bc962d43965e1d6 Mon Sep 17 00:00:00 2001 From: Sebastian Kunert Date: Tue, 13 May 2014 15:52:44 +0200 Subject: [PATCH 034/182] implemented parsing of the strings in SemanticPropUtil --- .../java/functions/FunctionAnnotation.java | 12 +- .../api/java/functions/SemanticPropUtil.java | 209 ++++++++++++++++-- .../translation/PlanFlatMapOperator.java | 5 +- 3 files changed, 202 insertions(+), 24 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java index a0883b1c28d93..6cb9181e94d5d 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java @@ -179,7 +179,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsExcept { - String[] value(); + String value(); } /** @@ -211,7 +211,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsFirstExcept { - String[] value(); + String value(); } @@ -244,7 +244,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsSecondExcept { - String[] value(); + String value(); } /** @@ -255,7 +255,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFields { - String[] value(); + String value(); } /** @@ -266,7 +266,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsSecond { - String[] value(); + String value(); } /** @@ -277,7 +277,7 @@ public class FunctionAnnotation { @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsFirst { - String[] value(); + String value(); } /** * Private constructor to prevent instantiation. This class is intended only as a container. diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java index 04477ff2f339d..6f6a7a52701fb 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java @@ -3,19 +3,37 @@ import java.lang.annotation.Annotation; import java.util.Iterator; import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import eu.stratosphere.api.common.operators.DualInputSemanticProperties; +import eu.stratosphere.api.common.operators.SemanticProperties; import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; +import eu.stratosphere.api.common.operators.util.FieldSet; import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFields; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsSecond; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsFirstExcept; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsSecondExcept; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ReadFieldsFirst; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ReadFieldsSecond; import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsExcept; import eu.stratosphere.api.java.functions.FunctionAnnotation.ReadFields; import eu.stratosphere.api.java.typeutils.TypeInformation; public class SemanticPropUtil { - public SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { + + private final static String REGEX_ANNOTATION = "\\s*(\\d+)\\s*->(\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))"; + + public static SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { Iterator it = set.iterator(); SingleInputSemanticProperties result = null; - + + //non tuple types are not yet supported for annotations + if (!inType.isTupleType() || !outType.isTupleType()) { + return null; + } + while (it.hasNext()) { if (result == null) { result = new SingleInputSemanticProperties(); @@ -25,28 +43,185 @@ public SingleInputSemanticProperties getSemanticPropsSingle(Set set, if (ann instanceof ConstantFields) { ConstantFields cf = (ConstantFields) ann; - } else if (ann instanceof ConstantFieldsExcept) { + parseConstantFields(cf.value(), result, inType, outType); + } else if (ann instanceof ConstantFieldsExcept) { ConstantFieldsExcept cfe = (ConstantFieldsExcept) ann; + parseConstantFieldsExcept(cfe.value(), result, inType, outType); } else if (ann instanceof ReadFields) { ReadFields rf = (ReadFields) ann; + parseReadFields(rf.value(), result, inType, outType); } } - return null; - } - - private void parseConstantFields(ConstantFields cf, SingleInputSemanticProperties sm) { - - } - - private void parseConstantFieldsExcept(ConstantFieldsExcept cfe, SingleInputSemanticProperties sm) { - + return result; } - private void parseReadFields(ReadFields rf, SingleInputSemanticProperties sm) { - + private static void parseConstantFields(String[] cf, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { + for (String s: cf) { + readConstantSet(sm, s, inType, outType, 0); + } + } + + private static void readConstantSet(SemanticProperties sp, String s, TypeInformation inType, TypeInformation outType, int input) { + Pattern check = Pattern.compile(REGEX_ANNOTATION); + Matcher matcher = check.matcher(s); + int sourceField = 0; + + if (!matcher.matches()) { + throw new RuntimeException("Wrong annotation String format. Please read the documentation."); + } + + sourceField = Integer.valueOf(matcher.group(1)); + if (!isValidField(inType, sourceField)) { + throw new IndexOutOfBoundsException("Annotation: Field " + sourceField + " not available in the input tuple."); + } + FieldSet fs = readFieldSetFromString(matcher.group(2), inType, outType); + + if (sp instanceof SingleInputSemanticProperties) { + ((SingleInputSemanticProperties) sp).addForwardedField(sourceField, fs); + } else if (sp instanceof DualInputSemanticProperties) { + if (input == 0) { + ((DualInputSemanticProperties) sp).addForwardedField1(sourceField, fs); + } else if (input == 1) { + ((DualInputSemanticProperties) sp).addForwardedField2(sourceField, fs); + } + } + } + + private static void parseConstantFieldsFirst(String[] cff, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + Pattern check = Pattern.compile(REGEX_ANNOTATION); + for (String s: cff) { + readConstantSet(dm, s, inType, outType, 0); + } + } + + private static void parseConstantFieldsSecond(String[] cfs, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + Pattern check = Pattern.compile(REGEX_ANNOTATION); + for (String s: cfs) { + readConstantSet(dm, s, inType, outType, 1); + } + } + + private static void parseConstantFieldsFirstExcept(String cffe, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + FieldSet fs = readFieldSetFromString(cffe, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + dm.addForwardedField1(i, i); + } + } + } + + private static void parseConstantFieldsSecondExcept(String cfse, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + FieldSet fs = readFieldSetFromString(cfse, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + dm.addForwardedField2(i, i); + } + } + } + + private static void parseReadFieldsFirst(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + FieldSet fs = readFieldSetFromString(rf, inType, outType); + dm.addReadFields1(fs); + } + + private static void parseReadFieldsSecond(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + FieldSet fs = readFieldSetFromString(rf, inType, outType); + dm.addReadFields2(fs); + } + + + private static boolean isValidField(TypeInformation type, int field) { + if (field > type.getArity() || field < 0) { + return false; + } + return true; + } + + private static void parseConstantFieldsExcept(String cfe, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { + FieldSet fs = readFieldSetFromString(cfe, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + sm.addForwardedField(i,i); + } + } + } + + private static FieldSet readFieldSetFromString(String s, TypeInformation inType, TypeInformation outType) { + Pattern check = Pattern.compile("\\s*(\\d+\\s*,\\s*)*(\\d+\\s*)"); + Pattern digit = Pattern.compile("\\d+"); + + Matcher matcher = check.matcher(s); + + if (!matcher.matches()) { + throw new RuntimeException("Wrong annotation String format. Please read the documentation."); + } + + matcher = digit.matcher(s); + FieldSet fs = new FieldSet(); + + while (matcher.find()) { + int field = Integer.valueOf(matcher.group()); + if (!isValidField(outType, field) || !isValidField(inType, field)) { + throw new IndexOutOfBoundsException("Annotation: Field " + field + " not available in the output tuple."); + } + fs.add(field); + } + return fs; + } + + private static void parseReadFields(String rf, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { + FieldSet fs = readFieldSetFromString(rf, inType, outType); + sm.addReadFields(fs); } - - public DualInputSemanticProperties getSemanticPropsDua(Set set, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { - return null; + + public static SingleInputSemanticProperties getSemanticPropsSingleFromString(String[] ConstantSet, String constantSetExcept, String ReadSet, TypeInformation inType, TypeInformation outType) { + return null; + } + + public static DualInputSemanticProperties getSemanticPropsDualFromString(String[] constantSetFirst, String[] constantSetSecond, String constantSetFirstExcept, + String constantSetSecondExcept, String readFieldsFirst, String readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { + return null; + } + + public static DualInputSemanticProperties getSemanticPropsDual(Set set, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { + Iterator it = set.iterator(); + DualInputSemanticProperties result = null; + + //non tuple types are not yet supported for annotations + if (!inType1.isTupleType() || !inType2.isTupleType() || !outType.isTupleType()) { + return null; + } + + while (it.hasNext()) { + if (result == null) { + result = new DualInputSemanticProperties(); + } + + Annotation ann = it.next(); + + if (ann instanceof ConstantFieldsFirst) { + ConstantFieldsFirst cff = (ConstantFieldsFirst) ann; + parseConstantFieldsFirst(cff.value(), result, inType1, outType); + } else if (ann instanceof ConstantFieldsSecond) { + ConstantFieldsSecond cfs = (ConstantFieldsSecond) ann; + parseConstantFieldsSecond(cfs.value(), result, inType2, outType); + } else if (ann instanceof ConstantFieldsFirstExcept) { + ConstantFieldsFirstExcept cffe = (ConstantFieldsFirstExcept) ann; + parseConstantFieldsFirstExcept(cffe.value(), result, inType1, outType); + } else if (ann instanceof ConstantFieldsSecondExcept) { + ConstantFieldsSecondExcept cfse = (ConstantFieldsSecondExcept) ann; + parseConstantFieldsSecondExcept(cfse.value(), result, inType2, outType); + } else if (ann instanceof ReadFieldsFirst) { + ReadFieldsFirst rff = (ReadFieldsFirst) ann; + parseReadFieldsFirst(rff.value(), result, inType1, outType); + } else if (ann instanceof ReadFieldsSecond) { + ReadFieldsSecond rfs = (ReadFieldsSecond) ann; + parseReadFieldsSecond(rfs.value(), result, inType2, outType); + } + } + return result; } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java index 147fbb3c7227b..99e98c06af662 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java @@ -18,10 +18,12 @@ import java.util.Set; import eu.stratosphere.api.common.functions.GenericFlatMap; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.common.operators.base.FlatMapOperatorBase; import eu.stratosphere.api.java.functions.FlatMapFunction; import eu.stratosphere.api.java.functions.FunctionAnnotation; import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFields; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; @@ -39,7 +41,8 @@ public PlanFlatMapOperator(FlatMapFunction udf, String name, TypeInformati this.outType = outType; Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); - System.out.println(annotations); + + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingle(annotations, this.inType, this.outType); } @Override From 2f13716ed57193f9c1d2aeaaefc70f4091b2eaf1 Mon Sep 17 00:00:00 2001 From: sebastian kunert Date: Wed, 7 May 2014 03:09:22 +0200 Subject: [PATCH 035/182] implemented semanticPropUtils class, added compatibility to operators --- .../java/functions/FunctionAnnotation.java | 282 ++++++------ .../api/java/functions/SemanticPropUtil.java | 407 ++++++++++-------- .../translation/PlanCogroupOperator.java | 16 +- .../translation/PlanCrossOperator.java | 22 +- .../translation/PlanFlatMapOperator.java | 21 +- .../translation/PlanGroupReduceOperator.java | 22 +- .../translation/PlanJoinOperator.java | 18 +- .../translation/PlanMapOperator.java | 18 +- .../translation/PlanReduceOperator.java | 9 +- .../semanticprops/SemanticPropUtilTest.java | 151 +++++++ 10 files changed, 607 insertions(+), 359 deletions(-) create mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java index 6cb9181e94d5d..45b1b73976012 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java @@ -13,71 +13,63 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.functions; +import eu.stratosphere.api.common.operators.util.UserCodeWrapper; + import java.lang.annotation.Annotation; import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import java.lang.annotation.Retention; import java.util.HashSet; import java.util.Set; -import com.google.common.primitives.Ints; - -import eu.stratosphere.api.common.operators.DualInputSemanticProperties; -import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; -import eu.stratosphere.api.common.operators.util.FieldSet; -import eu.stratosphere.api.common.operators.util.UserCodeWrapper; -import eu.stratosphere.api.java.typeutils.TypeInformation; - /** * This class defines the semantic assertions that can be added to functions. * The assertions are realized as java annotations, to be added to the class declaration of - * the class that realized the user function. For example, to declare the ConstantFieldsExcept - * annotation for a map-type function that realizes a simple absolute function, + * the class that realized the user function. For example, to declare the ConstantFields + * annotation for a map-type function that simply copies some fields, * use it the following way: - * + * *
          - * \@ConstantFieldsExcept(value={1,2}, outTuplePos={2,1}) - * public class MyMapper extends FlatMapFunction, Tuple3> + * \@ConstantFields({"0->0,1", "1->2"}) + * public class MyMapper extends FlatMapFunction, Tuple3> * { - * public void flatMap(Tuple3 value, Collector> out) { - Integer tmp = value.f2; - value.f2 = value.f1; - value.f1 = tmp; + * public void flatMap(Tuple3 value, Collector> out) { + value.f2 = value.f1 + value.f1 = value.f0; out.collect(value); } * } *
          - * - * Be aware that some annotations should only be used for functions with as single input - * ({@link MapFunction}, {@link ReduceFunction}) and some only for stubs with two inputs + * + * Be aware that some annotations should only be used for functions with as single input + * ({@link MapFunction}, {@link ReduceFunction}) and some only for stubs with two inputs * ({@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}). */ public class FunctionAnnotation { - + /** - * Specifies the fields of an input tuple or custom object that are unchanged in the output of + * Specifies the fields of an input tuple or custom object that are unchanged in the output of * a stub with a single input ( {@link MapFunction}, {@link ReduceFunction}). - * - * A field is considered to be constant if its value is not changed and copied to the same position of + * + * A field is considered to be constant if its value is not changed and copied to the same position of * output record. - * - * The annotation takes two int or String arrays. For correct use, one or two parameters should be set. The - * first array contains either integer positions of constant fields if tuples are used or the names of the fields - * for custom types. If only input positions are specified, it is assumed that the positions in the output remain identical. If - * a second parameter is set, it specifies the position of the values in the output data. - * + * + * The annotation takes one String array. The Strings represent the source and destination fields + * of the constant fields. The transition is represented by the string "->". The following would be a + * valid annotation "1->2,3". + * * * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be * inserted! Otherwise, the correct execution of a program can not be guaranteed. * So if in doubt, do not add a field to this set. * - * + * * This annotation is mutually exclusive with the {@link ConstantFieldsExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsExcept} annotation is not set, it is + * + * If this annotation and the {@link ConstantFieldsExcept} annotation is not set, it is * assumed that no field is constant. * */ @@ -86,31 +78,30 @@ public class FunctionAnnotation { public @interface ConstantFields { String[] value(); } - + /** - * Specifies the fields of an input tuple or custom object of the first input that are unchanged in + * Specifies the fields of an input tuple or custom object of the first input that are unchanged in * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) - * - * A field is considered to be constant if its value is not changed and copied to the same position of + * + * A field is considered to be constant if its value is not changed and copied to the same position of * output record. - * - * The annotation takes two int or String arrays. For correct use, one or two parameters should be set. The - * first array contains either integer positions of constant fields if tuples are used or the names of the fields - * for custom types. If only input positions are specified, it is assumed that the positions in the output remain identical. If - * a second parameter is set, it specifies the position of the values in the output data. - * + * + * The annotation takes one String array. The Strings represent the source and destination fields + * of the constant fields. The transition is represented by the string "->". The following would be a + * valid annotation "1->2,3". + * * * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be * inserted! Otherwise, the correct execution of a program can not be guaranteed. * So if in doubt, do not add a field to this set. * * * This annotation is mutually exclusive with the {@link ConstantFieldsFirstExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsFirstExcept} annotation is not set, it is + * + * If this annotation and the {@link ConstantFieldsFirstExcept} annotation is not set, it is * assumed that no field is constant. - * + * * */ @Target(ElementType.TYPE) @@ -118,61 +109,59 @@ public class FunctionAnnotation { public @interface ConstantFieldsFirst { String[] value(); } - + /** - * Specifies the fields of an input tuple or custom object of the second input that are unchanged in + * Specifies the fields of an input tuple or custom object of the second input that are unchanged in * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) - * - * A field is considered to be constant if its value is not changed and copied to the same position of + * + * A field is considered to be constant if its value is not changed and copied to the same position of * output record. - * - * The annotation takes two int or String arrays. For correct use, one or two parameters should be set. The - * first array contains either integer positions of constant fields if tuples are used or the names of the fields - * for custom types. If only input positions are specified, it is assumed that the positions in the output remain identical. If - * a second parameter is set, it specifies the position of the values in the output data. - * + * + * The annotation takes one String array. The Strings represent the source and destination fields + * of the constant fields. The transition is represented by the string "->". The following would be a + * valid annotation "1->2,3". + * * * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be * inserted! Otherwise, the correct execution of a program can not be guaranteed. * So if in doubt, do not add a field to this set. * * * This annotation is mutually exclusive with the {@link ConstantFieldsSecondExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsSecondExcept} annotation is not set, it is + * + * If this annotation and the {@link ConstantFieldsSecondExcept} annotation is not set, it is * assumed that no field is constant. - * + * */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsSecond { String[] value(); } - + /** - * Specifies the fields of an input tuple or custom object that are changed in the output of - * a stub with a single input ( {@link MapFunction}, {@link ReduceFunction}). All other + * Specifies the fields of an input tuple or custom object that are changed in the output of + * a stub with a single input ( {@link MapFunction}, {@link ReduceFunction}). All other * fields are assumed to be constant. - * - * A field is considered to be constant if its value is not changed and copied to the same position of + * + * A field is considered to be constant if its value is not changed and copied to the same position of * output record. - * - * The annotation takes one array specifying the positions of the input types that do not remain constant. This - * is possible for custom types using the 'inCustomPos' parameter and for tuples using the 'inTuplePos' parameter. + * + * The annotation takes one String array specifying the positions of the input types that do not remain constant. * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model * more complex situations use the \@ConstantFields annotation. - * + * * * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be * inserted! Otherwise, the correct execution of a program can not be guaranteed. * So if in doubt, do not add a field to this set. * - * + * * This annotation is mutually exclusive with the {@link ConstantFields} annotation. - * - * If this annotation and the {@link ConstantFields} annotation is not set, it is + * + * If this annotation and the {@link ConstantFields} annotation is not set, it is * assumed that no field is constant. * */ @@ -181,99 +170,97 @@ public class FunctionAnnotation { public @interface ConstantFieldsExcept { String value(); } - + /** - * Specifies the fields of an input tuple or custom object of the first input that are changed in + * Specifies the fields of an input tuple or custom object of the first input that are changed in * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) * All other fields are assumed to be constant. - * - * A field is considered to be constant if its value is not changed and copied to the same position of + * + * A field is considered to be constant if its value is not changed and copied to the same position of * output record. - * - * The annotation takes one array specifying the positions of the input types that do not remain constant. This - * is possible for custom types using the 'inCustomPos' parameter and for tuples using the 'inTuplePos' parameter. - * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model - * more complex situations use the \@ConstantFields annotation. - * + * + * The annotation takes one String array specifying the positions of the input types that do not remain constant. + * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model + * more complex situations use the \@ConstantFields annotation. + * * * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be * inserted! Otherwise, the correct execution of a program can not be guaranteed. * So if in doubt, do not add a field to this set. * * * This annotation is mutually exclusive with the {@link ConstantFieldsFirst} annotation. - * - * If this annotation and the {@link ConstantFieldsFirst} annotation is not set, it is + * + * If this annotation and the {@link ConstantFieldsFirst} annotation is not set, it is * assumed that no field is constant. - * + * */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsFirstExcept { String value(); } - - + + /** - * Specifies the fields of an input tuple or custom object of the second input that are changed in + * Specifies the fields of an input tuple or custom object of the second input that are changed in * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) * All other fields are assumed to be constant. - * - * A field is considered to be constant if its value is not changed and copied to the same position of + * + * A field is considered to be constant if its value is not changed and copied to the same position of * output record. - * - * The annotation takes one array specifying the positions of the input types that do not remain constant. This - * is possible for custom types using the 'inCustomPos' parameter and for tuples using the 'inTuplePos' parameter. - * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model - * more complex situations use the \@ConstantFields annotation. - * + * + * The annotation takes one String array specifying the positions of the input types that do not remain constant. + * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model + * more complex situations use the \@ConstantFields annotation. + * * * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be * inserted! Otherwise, the correct execution of a program can not be guaranteed. * So if in doubt, do not add a field to this set. * * * This annotation is mutually exclusive with the {@link ConstantFieldsSecond} annotation. - * - * If this annotation and the {@link ConstantFieldsSecond} annotation is not set, it is + * + * If this annotation and the {@link ConstantFieldsSecond} annotation is not set, it is * assumed that no field is constant. - * + * */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsSecondExcept { String value(); } - + /** - * Specifies the fields of an input tuple or custom object that are accessed in the function. This annotation should be used + * Specifies the fields of an input tuple that are accessed in the function. This annotation should be used * with user defined functions with one input. */ - + @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFields { String value(); } - + /** - * Specifies the fields of an input tuple or custom object that are accessed in the function. This annotation should be used + * Specifies the fields of an input tuple that are accessed in the function. This annotation should be used * with user defined functions with two inputs. */ - + @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsSecond { String value(); } - + /** - * Specifies the fields of an input tuple or custom object that are accessed in the function. This annotation should be used + * Specifies the fields of an input tuple that are accessed in the function. This annotation should be used * with user defined functions with two inputs. */ - + @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsFirst { @@ -283,106 +270,95 @@ public class FunctionAnnotation { * Private constructor to prevent instantiation. This class is intended only as a container. */ private FunctionAnnotation() {} - + // -------------------------------------------------------------------------------------------- // Function Annotation Handling // -------------------------------------------------------------------------------------------- - + /** * Reads the annotations of a user defined function with one input and returns semantic properties according to the constant fields annotated. * @param udf The user defined function. - * @param input Type information of the operator input. - * @param output Type information of the operator output. * @return The DualInputSemanticProperties containing the constant fields. */ - - public static Set readSingleConstantAnnotations(UserCodeWrapper udf) { + + public static Set readSingleConstantAnnotations(UserCodeWrapper udf) { ConstantFields constantSet = udf.getUserCodeAnnotation(ConstantFields.class); ConstantFieldsExcept notConstantSet = udf.getUserCodeAnnotation(ConstantFieldsExcept.class); ReadFields readfieldSet = udf.getUserCodeAnnotation(ReadFields.class); Set result = null; - + if (notConstantSet != null && constantSet != null) { throw new RuntimeException("Either ConstantFields or ConstantFieldsExcept can be specified, not both."); } - + if (notConstantSet != null) { - if (result == null) { result = new HashSet(); - } + result.add(notConstantSet); } if (constantSet != null) { - if (result == null) { result = new HashSet(); - } + result.add(constantSet); } - + if (readfieldSet != null) { if (result == null) { result = new HashSet(); } result.add(readfieldSet); } - + return result; } - + // -------------------------------------------------------------------------------------------- /** * Reads the annotations of a user defined function with two inputs and returns semantic properties according to the constant fields annotated. * @param udf The user defined function. - * @param input1 Type information of the first operator input. - * @param input2 Type information of the second operator input. - * @param output Type information of the operator output. * @return The DualInputSemanticProperties containing the constant fields. */ - + public static Set readDualConstantAnnotations(UserCodeWrapper udf) { // get readSet annotation from stub ConstantFieldsFirst constantSet1 = udf.getUserCodeAnnotation(ConstantFieldsFirst.class); ConstantFieldsSecond constantSet2= udf.getUserCodeAnnotation(ConstantFieldsSecond.class); - + // get readSet annotation from stub ConstantFieldsFirstExcept notConstantSet1 = udf.getUserCodeAnnotation(ConstantFieldsFirstExcept.class); ConstantFieldsSecondExcept notConstantSet2 = udf.getUserCodeAnnotation(ConstantFieldsSecondExcept.class); - + ReadFieldsFirst readfieldSet1 = udf.getUserCodeAnnotation(ReadFieldsFirst.class); ReadFieldsSecond readfieldSet2 = udf.getUserCodeAnnotation(ReadFieldsSecond.class); - + if (notConstantSet1 != null && constantSet1 != null) { throw new RuntimeException("Either ConstantFieldsFirst or ConstantFieldsFirstExcept can be specified, not both."); } - + if (constantSet2 != null && notConstantSet2 != null) { throw new RuntimeException("Either ConstantFieldsSecond or ConstantFieldsSecondExcept can be specified, not both."); } - + Set result = null; - + if (notConstantSet2 != null) { - if (result == null) { - result = new HashSet(); - } + result = new HashSet(); result.add(notConstantSet2); } if (constantSet2 != null) { - if (result == null) { - result = new HashSet(); - } + result = new HashSet(); result.add(constantSet2); } - + if (readfieldSet2 != null) { if (result == null) { result = new HashSet(); } result.add(readfieldSet2); } - + if (notConstantSet1 != null) { if (result == null) { result = new HashSet(); @@ -395,19 +371,19 @@ public static Set readDualConstantAnnotations(UserCodeWrapper udf } result.add(constantSet1); } - + if (readfieldSet1 != null) { if (result == null) { result = new HashSet(); } result.add(readfieldSet1); } - + return result; } - - - - + + + + } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java index 6f6a7a52701fb..c0bb5c4b47963 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java @@ -1,3 +1,16 @@ +/*********************************************************************************************************************** + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.functions; import java.lang.annotation.Annotation; @@ -23,205 +36,257 @@ public class SemanticPropUtil { - private final static String REGEX_ANNOTATION = "\\s*(\\d+)\\s*->(\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))"; + private final static String REGEX_ANNOTATION = "\\s*(\\d+)\\s*->(\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))"; - public static SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { + public static SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { + if (set == null) { + return null; + } Iterator it = set.iterator(); SingleInputSemanticProperties result = null; - //non tuple types are not yet supported for annotations - if (!inType.isTupleType() || !outType.isTupleType()) { - return null; - } + //non tuple types are not yet supported for annotations + if (!inType.isTupleType() || !outType.isTupleType()) { + return null; + } while (it.hasNext()) { if (result == null) { result = new SingleInputSemanticProperties(); } - + Annotation ann = it.next(); - + if (ann instanceof ConstantFields) { ConstantFields cf = (ConstantFields) ann; - parseConstantFields(cf.value(), result, inType, outType); - } else if (ann instanceof ConstantFieldsExcept) { + parseConstantFields(cf.value(), result, inType, outType); + } else if (ann instanceof ConstantFieldsExcept) { ConstantFieldsExcept cfe = (ConstantFieldsExcept) ann; - parseConstantFieldsExcept(cfe.value(), result, inType, outType); + parseConstantFieldsExcept(cfe.value(), result, inType, outType); } else if (ann instanceof ReadFields) { ReadFields rf = (ReadFields) ann; - parseReadFields(rf.value(), result, inType, outType); + parseReadFields(rf.value(), result, inType, outType); } } return result; } - + private static void parseConstantFields(String[] cf, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { - for (String s: cf) { - readConstantSet(sm, s, inType, outType, 0); - } - } - - private static void readConstantSet(SemanticProperties sp, String s, TypeInformation inType, TypeInformation outType, int input) { - Pattern check = Pattern.compile(REGEX_ANNOTATION); - Matcher matcher = check.matcher(s); - int sourceField = 0; - - if (!matcher.matches()) { - throw new RuntimeException("Wrong annotation String format. Please read the documentation."); - } - - sourceField = Integer.valueOf(matcher.group(1)); - if (!isValidField(inType, sourceField)) { - throw new IndexOutOfBoundsException("Annotation: Field " + sourceField + " not available in the input tuple."); - } - FieldSet fs = readFieldSetFromString(matcher.group(2), inType, outType); - - if (sp instanceof SingleInputSemanticProperties) { - ((SingleInputSemanticProperties) sp).addForwardedField(sourceField, fs); - } else if (sp instanceof DualInputSemanticProperties) { - if (input == 0) { - ((DualInputSemanticProperties) sp).addForwardedField1(sourceField, fs); - } else if (input == 1) { - ((DualInputSemanticProperties) sp).addForwardedField2(sourceField, fs); - } - } - } - - private static void parseConstantFieldsFirst(String[] cff, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { - Pattern check = Pattern.compile(REGEX_ANNOTATION); - for (String s: cff) { - readConstantSet(dm, s, inType, outType, 0); - } - } - - private static void parseConstantFieldsSecond(String[] cfs, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { - Pattern check = Pattern.compile(REGEX_ANNOTATION); - for (String s: cfs) { - readConstantSet(dm, s, inType, outType, 1); - } - } - - private static void parseConstantFieldsFirstExcept(String cffe, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { - FieldSet fs = readFieldSetFromString(cffe, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - dm.addForwardedField1(i, i); - } - } - } - - private static void parseConstantFieldsSecondExcept(String cfse, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { - FieldSet fs = readFieldSetFromString(cfse, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - dm.addForwardedField2(i, i); - } - } - } - - private static void parseReadFieldsFirst(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { - FieldSet fs = readFieldSetFromString(rf, inType, outType); - dm.addReadFields1(fs); - } - - private static void parseReadFieldsSecond(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { - FieldSet fs = readFieldSetFromString(rf, inType, outType); - dm.addReadFields2(fs); - } - - - private static boolean isValidField(TypeInformation type, int field) { - if (field > type.getArity() || field < 0) { - return false; - } - return true; - } + if (cf == null) { + return; + } + for (String s : cf) { + readConstantSet(sm, s, inType, outType, 0); + } + } + + private static void readConstantSet(SemanticProperties sp, String s, TypeInformation inType, TypeInformation outType, int input) { + Pattern check = Pattern.compile(REGEX_ANNOTATION); + Matcher matcher = check.matcher(s); + int sourceField = 0; + + if (!matcher.matches()) { + throw new RuntimeException("Wrong annotation String format. Please read the documentation."); + } + + sourceField = Integer.valueOf(matcher.group(1)); + if (!isValidField(inType, sourceField)) { + throw new IndexOutOfBoundsException("Annotation: Field " + sourceField + " not available in the input tuple."); + } + FieldSet fs = readFieldSetFromString(matcher.group(2), inType, outType); + + if (sp instanceof SingleInputSemanticProperties) { + ((SingleInputSemanticProperties) sp).addForwardedField(sourceField, fs); + } else if (sp instanceof DualInputSemanticProperties) { + if (input == 0) { + ((DualInputSemanticProperties) sp).addForwardedField1(sourceField, fs); + } else if (input == 1) { + ((DualInputSemanticProperties) sp).addForwardedField2(sourceField, fs); + } + } + } + + private static void parseConstantFieldsFirst(String[] cff, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + if (cff == null) { + return; + } + + Pattern check = Pattern.compile(REGEX_ANNOTATION); + for (String s : cff) { + readConstantSet(dm, s, inType, outType, 0); + } + } + + private static void parseConstantFieldsSecond(String[] cfs, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + if (cfs == null) { + return; + } + Pattern check = Pattern.compile(REGEX_ANNOTATION); + for (String s : cfs) { + readConstantSet(dm, s, inType, outType, 1); + } + } + + private static void parseConstantFieldsFirstExcept(String cffe, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + if (cffe == null) { + return; + } + + FieldSet fs = readFieldSetFromString(cffe, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + dm.addForwardedField1(i, i); + } + } + } + + private static void parseConstantFieldsSecondExcept(String cfse, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + if (cfse == null) { + return; + } + + FieldSet fs = readFieldSetFromString(cfse, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + dm.addForwardedField2(i, i); + } + } + } + + private static void parseReadFieldsFirst(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + if (rf == null) { + return; + } + + FieldSet fs = readFieldSetFromString(rf, inType, outType); + dm.addReadFields1(fs); + } + + private static void parseReadFieldsSecond(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + if (rf == null) { + return; + } + + FieldSet fs = readFieldSetFromString(rf, inType, outType); + dm.addReadFields2(fs); + } + + + private static boolean isValidField(TypeInformation type, int field) { + if (field > type.getArity() || field < 0) { + return false; + } + return true; + } private static void parseConstantFieldsExcept(String cfe, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { - FieldSet fs = readFieldSetFromString(cfe, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - sm.addForwardedField(i,i); - } - } - } - - private static FieldSet readFieldSetFromString(String s, TypeInformation inType, TypeInformation outType) { - Pattern check = Pattern.compile("\\s*(\\d+\\s*,\\s*)*(\\d+\\s*)"); - Pattern digit = Pattern.compile("\\d+"); - - Matcher matcher = check.matcher(s); - - if (!matcher.matches()) { - throw new RuntimeException("Wrong annotation String format. Please read the documentation."); - } - - matcher = digit.matcher(s); - FieldSet fs = new FieldSet(); - - while (matcher.find()) { - int field = Integer.valueOf(matcher.group()); - if (!isValidField(outType, field) || !isValidField(inType, field)) { - throw new IndexOutOfBoundsException("Annotation: Field " + field + " not available in the output tuple."); - } - fs.add(field); - } - return fs; - } + if (cfe == null) { + return; + } + + FieldSet fs = readFieldSetFromString(cfe, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + sm.addForwardedField(i, i); + } + } + } + + private static FieldSet readFieldSetFromString(String s, TypeInformation inType, TypeInformation outType) { + Pattern check = Pattern.compile("\\s*(\\d+\\s*,\\s*)*(\\d+\\s*)"); + Pattern digit = Pattern.compile("\\d+"); + + Matcher matcher = check.matcher(s); + + if (!matcher.matches()) { + throw new RuntimeException("Wrong annotation String format. Please read the documentation."); + } + + matcher = digit.matcher(s); + FieldSet fs = new FieldSet(); + + while (matcher.find()) { + int field = Integer.valueOf(matcher.group()); + if (!isValidField(outType, field) || !isValidField(inType, field)) { + throw new IndexOutOfBoundsException("Annotation: Field " + field + " not available in the output tuple."); + } + fs.add(field); + } + return fs; + } private static void parseReadFields(String rf, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { - FieldSet fs = readFieldSetFromString(rf, inType, outType); - sm.addReadFields(fs); + if (rf == null) { + return; + } + + FieldSet fs = readFieldSetFromString(rf, inType, outType); + sm.addReadFields(fs); } - public static SingleInputSemanticProperties getSemanticPropsSingleFromString(String[] ConstantSet, String constantSetExcept, String ReadSet, TypeInformation inType, TypeInformation outType) { - return null; - } + public static SingleInputSemanticProperties getSemanticPropsSingleFromString(String[] constantSet, String constantSetExcept, String readSet, TypeInformation inType, TypeInformation outType) { + SingleInputSemanticProperties result = new SingleInputSemanticProperties(); + parseConstantFields(constantSet, result, inType, outType); + parseConstantFieldsExcept(constantSetExcept, result, inType, outType); + parseReadFields(readSet, result, inType, outType); + return result; + } - public static DualInputSemanticProperties getSemanticPropsDualFromString(String[] constantSetFirst, String[] constantSetSecond, String constantSetFirstExcept, - String constantSetSecondExcept, String readFieldsFirst, String readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { - return null; - } + public static DualInputSemanticProperties getSemanticPropsDualFromString(String[] constantSetFirst, String[] constantSetSecond, String constantSetFirstExcept, + String constantSetSecondExcept, String readFieldsFirst, String readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { + DualInputSemanticProperties result = new DualInputSemanticProperties(); + parseConstantFieldsFirst(constantSetFirst, result, inType1, outType); + parseConstantFieldsSecond(constantSetSecond, result, inType2, outType); + parseConstantFieldsFirstExcept(constantSetFirstExcept, result, inType1, outType); + parseConstantFieldsSecondExcept(constantSetSecondExcept, result, inType2, outType); + parseReadFieldsFirst(readFieldsFirst, result, inType1, outType); + parseReadFieldsSecond(readFieldsSecond, result, inType2, outType); + return result; + } public static DualInputSemanticProperties getSemanticPropsDual(Set set, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { - Iterator it = set.iterator(); - DualInputSemanticProperties result = null; - - //non tuple types are not yet supported for annotations - if (!inType1.isTupleType() || !inType2.isTupleType() || !outType.isTupleType()) { - return null; - } - - while (it.hasNext()) { - if (result == null) { - result = new DualInputSemanticProperties(); - } - - Annotation ann = it.next(); - - if (ann instanceof ConstantFieldsFirst) { - ConstantFieldsFirst cff = (ConstantFieldsFirst) ann; - parseConstantFieldsFirst(cff.value(), result, inType1, outType); - } else if (ann instanceof ConstantFieldsSecond) { - ConstantFieldsSecond cfs = (ConstantFieldsSecond) ann; - parseConstantFieldsSecond(cfs.value(), result, inType2, outType); - } else if (ann instanceof ConstantFieldsFirstExcept) { - ConstantFieldsFirstExcept cffe = (ConstantFieldsFirstExcept) ann; - parseConstantFieldsFirstExcept(cffe.value(), result, inType1, outType); - } else if (ann instanceof ConstantFieldsSecondExcept) { - ConstantFieldsSecondExcept cfse = (ConstantFieldsSecondExcept) ann; - parseConstantFieldsSecondExcept(cfse.value(), result, inType2, outType); - } else if (ann instanceof ReadFieldsFirst) { - ReadFieldsFirst rff = (ReadFieldsFirst) ann; - parseReadFieldsFirst(rff.value(), result, inType1, outType); - } else if (ann instanceof ReadFieldsSecond) { - ReadFieldsSecond rfs = (ReadFieldsSecond) ann; - parseReadFieldsSecond(rfs.value(), result, inType2, outType); - } - } - return result; + if (set == null) { + return null; + } + + Iterator it = set.iterator(); + DualInputSemanticProperties result = null; + + //non tuple types are not yet supported for annotations + if (!inType1.isTupleType() || !inType2.isTupleType() || !outType.isTupleType()) { + return null; + } + + while (it.hasNext()) { + if (result == null) { + result = new DualInputSemanticProperties(); + } + + Annotation ann = it.next(); + + if (ann instanceof ConstantFieldsFirst) { + ConstantFieldsFirst cff = (ConstantFieldsFirst) ann; + parseConstantFieldsFirst(cff.value(), result, inType1, outType); + } else if (ann instanceof ConstantFieldsSecond) { + ConstantFieldsSecond cfs = (ConstantFieldsSecond) ann; + parseConstantFieldsSecond(cfs.value(), result, inType2, outType); + } else if (ann instanceof ConstantFieldsFirstExcept) { + ConstantFieldsFirstExcept cffe = (ConstantFieldsFirstExcept) ann; + parseConstantFieldsFirstExcept(cffe.value(), result, inType1, outType); + } else if (ann instanceof ConstantFieldsSecondExcept) { + ConstantFieldsSecondExcept cfse = (ConstantFieldsSecondExcept) ann; + parseConstantFieldsSecondExcept(cfse.value(), result, inType2, outType); + } else if (ann instanceof ReadFieldsFirst) { + ReadFieldsFirst rff = (ReadFieldsFirst) ann; + parseReadFieldsFirst(rff.value(), result, inType1, outType); + } else if (ann instanceof ReadFieldsSecond) { + ReadFieldsSecond rfs = (ReadFieldsSecond) ann; + parseReadFieldsSecond(rfs.value(), result, inType2, outType); + } + } + return result; } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCogroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCogroupOperator.java index 150d7c10b4a7e..7ffce3a3c7022 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCogroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCogroupOperator.java @@ -15,14 +15,20 @@ package eu.stratosphere.api.java.operators.translation; import eu.stratosphere.api.common.functions.GenericCoGrouper; +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; import eu.stratosphere.api.common.operators.base.CoGroupOperatorBase; import eu.stratosphere.api.java.functions.CoGroupFunction; +import eu.stratosphere.api.java.functions.FunctionAnnotation; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; -public class PlanCogroupOperator +import java.lang.annotation.Annotation; +import java.util.Set; + +public class PlanCogroupOperator extends CoGroupOperatorBase> implements BinaryJavaPlanNode { - + private final TypeInformation inType1; private final TypeInformation inType2; private final TypeInformation outType; @@ -31,10 +37,14 @@ public PlanCogroupOperator( CoGroupFunction udf, int[] keyPositions1, int[] keyPositions2, String name, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { super(udf, keyPositions1, keyPositions2, name); - + this.inType1 = inType1; this.inType2 = inType2; this.outType = outType; + + Set annotations = FunctionAnnotation.readDualConstantAnnotations(this.getUserCodeWrapper()); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDual(annotations, this.getInputType1(), this.getInputType2(), this.getReturnType()); + this.setSemanticProperties(dsp); } @Override diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCrossOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCrossOperator.java index 5cadbf0e8de79..998df94ec1e3a 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCrossOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanCrossOperator.java @@ -15,31 +15,41 @@ package eu.stratosphere.api.java.operators.translation; import eu.stratosphere.api.common.functions.GenericCrosser; +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; import eu.stratosphere.api.common.operators.base.CrossOperatorBase; import eu.stratosphere.api.java.functions.CrossFunction; +import eu.stratosphere.api.java.functions.FunctionAnnotation; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; -public class PlanCrossOperator +import java.lang.annotation.Annotation; +import java.util.Set; + +public class PlanCrossOperator extends CrossOperatorBase> implements BinaryJavaPlanNode{ - + private final TypeInformation inType1; private final TypeInformation inType2; private final TypeInformation outType; - + public PlanCrossOperator( CrossFunction udf, String name, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { super(udf, name); - + this.inType1 = inType1; this.inType2 = inType2; this.outType = outType; - + + Set annotations = FunctionAnnotation.readDualConstantAnnotations(this.getUserCodeWrapper()); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDual(annotations, this.getInputType1(), this.getInputType2(), this.getReturnType()); + this.setSemanticProperties(dsp); + } - + @Override public TypeInformation getReturnType() { return this.outType; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java index 99e98c06af662..3c89beeddeec8 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanFlatMapOperator.java @@ -14,37 +14,36 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.operators.translation; -import java.lang.annotation.Annotation; -import java.util.Set; - import eu.stratosphere.api.common.functions.GenericFlatMap; import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.common.operators.base.FlatMapOperatorBase; import eu.stratosphere.api.java.functions.FlatMapFunction; import eu.stratosphere.api.java.functions.FunctionAnnotation; -import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFields; import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; +import java.lang.annotation.Annotation; +import java.util.Set; + public class PlanFlatMapOperator extends FlatMapOperatorBase> implements UnaryJavaPlanNode { private final TypeInformation inType; - + private final TypeInformation outType; - - + + public PlanFlatMapOperator(FlatMapFunction udf, String name, TypeInformation inType, TypeInformation outType) { super(udf, name); this.inType = inType; this.outType = outType; - - Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingle(annotations, this.inType, this.outType); + Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingle(annotations, this.getInputType(), this.getReturnType()); + setSemanticProperties(sp); } - + @Override public TypeInformation getReturnType() { return this.outType; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanGroupReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanGroupReduceOperator.java index ca0376709f503..eceb496097a71 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanGroupReduceOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanGroupReduceOperator.java @@ -15,11 +15,17 @@ package eu.stratosphere.api.java.operators.translation; import eu.stratosphere.api.common.functions.GenericGroupReduce; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.common.operators.base.GroupReduceOperatorBase; +import eu.stratosphere.api.java.functions.FunctionAnnotation; import eu.stratosphere.api.java.functions.GroupReduceFunction; import eu.stratosphere.api.java.functions.GroupReduceFunction.Combinable; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; +import java.lang.annotation.Annotation; +import java.util.Set; + /** * */ @@ -28,20 +34,24 @@ public class PlanGroupReduceOperator extends GroupReduceOperatorBase inType; - + private final TypeInformation outType; - - - public PlanGroupReduceOperator(GroupReduceFunction udf, int[] logicalGroupingFields, String name, + + + public PlanGroupReduceOperator(GroupReduceFunction udf, int[] logicalGroupingFields, String name, TypeInformation inputType, TypeInformation outputType) { super(udf, logicalGroupingFields, name); - + this.inType = inputType; this.outType = outputType; super.setCombinable(getUserCodeWrapper().getUserCodeAnnotation(Combinable.class) != null); + + Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingle(annotations, this.inType, this.outType); + setSemanticProperties(sp); } - + @Override public TypeInformation getReturnType() { return this.outType; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanJoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanJoinOperator.java index b9d5886a81e04..c60daae4001c9 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanJoinOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanJoinOperator.java @@ -15,14 +15,20 @@ package eu.stratosphere.api.java.operators.translation; import eu.stratosphere.api.common.functions.GenericJoiner; +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; import eu.stratosphere.api.common.operators.base.JoinOperatorBase; +import eu.stratosphere.api.java.functions.FunctionAnnotation; import eu.stratosphere.api.java.functions.JoinFunction; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; -public class PlanJoinOperator +import java.lang.annotation.Annotation; +import java.util.Set; + +public class PlanJoinOperator extends JoinOperatorBase> implements BinaryJavaPlanNode { - + private final TypeInformation inType1; private final TypeInformation inType2; private final TypeInformation outType; @@ -31,12 +37,16 @@ public PlanJoinOperator( JoinFunction udf, int[] keyPositions1, int[] keyPositions2, String name, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { super(udf, keyPositions1, keyPositions2, name); - + this.inType1 = inType1; this.inType2 = inType2; this.outType = outType; + + Set annotations = FunctionAnnotation.readDualConstantAnnotations(this.getUserCodeWrapper()); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDual(annotations, this.getInputType1(), this.getInputType2(), this.getReturnType()); + this.setSemanticProperties(dsp); } - + @Override public TypeInformation getReturnType() { return this.outType; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanMapOperator.java index 5d0e2c12ec9b6..4939dba673ce4 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanMapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanMapOperator.java @@ -15,10 +15,16 @@ package eu.stratosphere.api.java.operators.translation; import eu.stratosphere.api.common.functions.GenericMap; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.common.operators.base.PlainMapOperatorBase; +import eu.stratosphere.api.java.functions.FunctionAnnotation; import eu.stratosphere.api.java.functions.MapFunction; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; +import java.lang.annotation.Annotation; +import java.util.Set; + /** * */ @@ -27,16 +33,20 @@ public class PlanMapOperator extends PlainMapOperatorBase { private final TypeInformation inType; - + private final TypeInformation outType; - - + + public PlanMapOperator(MapFunction udf, String name, TypeInformation inType, TypeInformation outType) { super(udf, name); this.inType = inType; this.outType = outType; + + Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingle(annotations, this.inType, this.outType); + setSemanticProperties(sp); } - + @Override public TypeInformation getReturnType() { return this.outType; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java index 59fce2d36c6cd..5838835bea7da 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java @@ -17,8 +17,12 @@ import eu.stratosphere.api.common.functions.GenericReduce; import eu.stratosphere.api.common.operators.base.ReduceOperatorBase; import eu.stratosphere.api.java.functions.ReduceFunction; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; +import java.lang.annotation.Annotation; +import java.util.Set; + /** * */ @@ -32,6 +36,10 @@ public class PlanReduceOperator extends ReduceOperatorBase> public PlanReduceOperator(ReduceFunction udf, int[] logicalGroupingFields, String name, TypeInformation type) { super(udf, logicalGroupingFields, name); this.type = type; + + Set annotations = FunctionAnnotation.readSingleConstantAnnotations(this.getUserCodeWrapper()); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingle(annotations, this.getInputType(), this.getReturnType()); + setSemanticProperties(sp); } @@ -44,5 +52,4 @@ public TypeInformation getReturnType() { public TypeInformation getInputType() { return this.type; } - } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java new file mode 100644 index 0000000000000..687d08d61cde6 --- /dev/null +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java @@ -0,0 +1,151 @@ +/*********************************************************************************************************************** + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.test.semanticprops; + +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; +import eu.stratosphere.api.common.operators.util.FieldSet; +import eu.stratosphere.api.java.functions.SemanticPropUtil; +import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.api.java.typeutils.BasicTypeInfo; +import eu.stratosphere.api.java.typeutils.TupleTypeInfo; +import eu.stratosphere.api.java.typeutils.TypeInformation; +import junit.framework.Assert; +import org.junit.Test; + +public class SemanticPropUtilTest { + + @Test + public void testSimpleCase() { + String[] constantFields = {"1->1,2", "2->3"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(3)); + } + + @Test + public void testConstantFieldsExcept() { + String constantFieldsExcept = "1"; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(null, constantFieldsExcept, null,type, type); + + FieldSet fs = sp.getForwardedField(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(0)); + + fs = sp.getForwardedField(1); + Assert.assertTrue(fs == null); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(2)); + } + + @Test + public void testReadFields() { + String readFields = "1, 2"; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(null, null, readFields,type, type); + + FieldSet fs = sp.getReadFields(); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(2)); + Assert.assertTrue(fs.contains(1)); + } + + @Test + public void testSimpleCaseDual() { + String[] constantFieldsFirst = {"1->1,2", "2->3"}; + String[] constantFieldsSecond = {"1->1,2", "2->3"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(constantFieldsFirst, constantFieldsSecond, null, null, null, null, type, type, type); + + FieldSet fs = dsp.getForwardedField1(1); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + + fs = dsp.getForwardedField1(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(3)); + } + + @Test + public void testFieldsExceptDual() { + String constantFieldsFirstExcept = "1,2"; + String[] constantFieldsSecond = {"0->1"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(null, constantFieldsSecond, constantFieldsFirstExcept, null, null, null, type, type, type); + + FieldSet fs = dsp.getForwardedField1(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(0)); + + fs = dsp.getForwardedField1(1); + Assert.assertTrue(fs == null); + + fs = dsp.getForwardedField1(2); + Assert.assertTrue(fs == null); + + fs = dsp.getForwardedField2(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(1)); + } + + @Test + public void testStringParse1() { + String[] constantFields = {" 1-> 1 , 2", "2 ->3"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(3)); + } + + @Test + public void testStringParse2() { + String[] constantFields = {"notValid"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + + try { + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); + } catch (Exception e) { + return; + } + Assert.fail(); + } + + +} From 4af8ffa236d6260263e5a3e82eb88968a698cec9 Mon Sep 17 00:00:00 2001 From: sebastian kunert Date: Wed, 14 May 2014 00:48:23 +0200 Subject: [PATCH 036/182] Added Support for Wildcards. The following is now possible "*" for all fields constant and "1->*" when the first field is forwarded to all output fields. --- .../api/java/functions/SemanticPropUtil.java | 41 ++- .../api/java/operators/CoGroupOperator.java | 192 ++++++------- .../api/java/operators/CrossOperator.java | 266 +++++++++--------- .../api/java/operators/FlatMapOperator.java | 18 +- .../api/java/operators/JoinOperator.java | 4 + .../api/java/operators/MapOperator.java | 20 +- .../java/operators/ReduceGroupOperator.java | 4 + .../api/java/operators/ReduceOperator.java | 5 + .../operators/SingleInputUdfOperator.java | 49 ++-- .../java/operators/TwoInputUdfOperator.java | 53 ++-- .../semanticprops/SemanticPropUtilTest.java | 36 +++ 11 files changed, 404 insertions(+), 284 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java index c0bb5c4b47963..b8d6de76403e4 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java @@ -36,7 +36,7 @@ public class SemanticPropUtil { - private final static String REGEX_ANNOTATION = "\\s*(\\d+)\\s*->(\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))"; + private final static String REGEX_ANNOTATION = "\\s*(\\d+)\\s*->((\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))|(\\*))"; public static SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { if (set == null) { @@ -81,6 +81,23 @@ private static void parseConstantFields(String[] cf, SingleInputSemanticProperti } private static void readConstantSet(SemanticProperties sp, String s, TypeInformation inType, TypeInformation outType, int input) { + if (s.equals("*")) { + if (sp instanceof SingleInputSemanticProperties) { + for (int i = 0; i < inType.getArity() && i < outType.getArity(); i++) { + ((SingleInputSemanticProperties) sp).addForwardedField(i, i); + } + } else if (sp instanceof DualInputSemanticProperties) { + for (int i = 0; i < inType.getArity() && i < outType.getArity(); i++) { + if (input == 0) { + ((DualInputSemanticProperties) sp).addForwardedField1(i, i); + } else if (input == 1) { + ((DualInputSemanticProperties) sp).addForwardedField2(i, i); + } + } + } + return; + } + Pattern check = Pattern.compile(REGEX_ANNOTATION); Matcher matcher = check.matcher(s); int sourceField = 0; @@ -93,7 +110,25 @@ private static void readConstantSet(SemanticProperties sp, String s, TypeInforma if (!isValidField(inType, sourceField)) { throw new IndexOutOfBoundsException("Annotation: Field " + sourceField + " not available in the input tuple."); } - FieldSet fs = readFieldSetFromString(matcher.group(2), inType, outType); + + if (matcher.group(6) != null) { + if (sp instanceof SingleInputSemanticProperties) { + for (int i = 0; i < outType.getArity(); i++) { + ((SingleInputSemanticProperties) sp).addForwardedField(sourceField, i); + } + } else if (sp instanceof DualInputSemanticProperties) { + for (int i = 0; i < outType.getArity(); i++) { + if (input == 0) { + ((DualInputSemanticProperties) sp).addForwardedField1(sourceField, i); + } else if (input == 1) { + ((DualInputSemanticProperties) sp).addForwardedField2(sourceField, i); + } + } + } + return; + } + + FieldSet fs = readFieldSetFromString(matcher.group(3), inType, outType); if (sp instanceof SingleInputSemanticProperties) { ((SingleInputSemanticProperties) sp).addForwardedField(sourceField, fs); @@ -236,7 +271,7 @@ public static SingleInputSemanticProperties getSemanticPropsSingleFromString(Str } public static DualInputSemanticProperties getSemanticPropsDualFromString(String[] constantSetFirst, String[] constantSetSecond, String constantSetFirstExcept, - String constantSetSecondExcept, String readFieldsFirst, String readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { + String constantSetSecondExcept, String readFieldsFirst, String readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { DualInputSemanticProperties result = new DualInputSemanticProperties(); parseConstantFieldsFirst(constantSetFirst, result, inType1, outType); parseConstantFieldsSecond(constantSetSecond, result, inType2, outType); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java index 73018b3e5d754..d3d17c43851a2 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java @@ -35,7 +35,7 @@ * */ public class CoGroupOperator extends TwoInputUdfOperator> { - + private final CoGroupFunction function; private final Keys keys1; @@ -54,111 +54,115 @@ protected CoGroupOperator(DataSet input1, DataSet input2, if (keys1 == null || keys2 == null) { throw new NullPointerException(); } - + this.keys1 = keys1; this.keys2 = keys2; } - + protected Keys getKeys1() { return this.keys1; } - + protected Keys getKeys2() { return this.keys2; } - + @Override protected Operator translateToDataFlow(Operator input1, Operator input2) { - + String name = getName() != null ? getName() : function.getClass().getName(); - - if (keys1 instanceof Keys.SelectorFunctionKeys + + if (keys1 instanceof Keys.SelectorFunctionKeys && keys2 instanceof Keys.SelectorFunctionKeys && keys1.areCompatibale(keys2)) { - + @SuppressWarnings("unchecked") Keys.SelectorFunctionKeys selectorKeys1 = (Keys.SelectorFunctionKeys) keys1; @SuppressWarnings("unchecked") Keys.SelectorFunctionKeys selectorKeys2 = (Keys.SelectorFunctionKeys) keys2; - - PlanUnwrappingCoGroupOperator po = - translateSelectorFunctionCoGroup(selectorKeys1, selectorKeys2, function, + + PlanUnwrappingCoGroupOperator po = + translateSelectorFunctionCoGroup(selectorKeys1, selectorKeys2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - + // set dop po.setDegreeOfParallelism(this.getParallelism()); - + return po; - + } - else if (keys1 instanceof Keys.FieldPositionKeys - && keys2 instanceof Keys.FieldPositionKeys + else if (keys1 instanceof Keys.FieldPositionKeys + && keys2 instanceof Keys.FieldPositionKeys && keys1.areCompatibale(keys2) ) { - + int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions(); int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions(); - - PlanCogroupOperator po = - new PlanCogroupOperator(function, logicalKeyPositions1, logicalKeyPositions2, + + PlanCogroupOperator po = + new PlanCogroupOperator(function, logicalKeyPositions1, logicalKeyPositions2, name, getInput1Type(), getInput2Type(), getResultType()); - + // set inputs po.setFirstInput(input1); po.setSecondInput(input2); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } // set dop po.setDegreeOfParallelism(this.getParallelism()); - + return po; - + } - else if (keys1 instanceof Keys.FieldPositionKeys + else if (keys1 instanceof Keys.FieldPositionKeys && keys2 instanceof Keys.SelectorFunctionKeys && keys1.areCompatibale(keys2) ) { - + int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions(); - + @SuppressWarnings("unchecked") Keys.SelectorFunctionKeys selectorKeys2 = (Keys.SelectorFunctionKeys) keys2; - - PlanUnwrappingCoGroupOperator po = - translateSelectorFunctionCoGroupRight(logicalKeyPositions1, selectorKeys2, function, + + PlanUnwrappingCoGroupOperator po = + translateSelectorFunctionCoGroupRight(logicalKeyPositions1, selectorKeys2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - + // set dop po.setDegreeOfParallelism(this.getParallelism()); - + return po; } else if (keys1 instanceof Keys.SelectorFunctionKeys - && keys2 instanceof Keys.FieldPositionKeys + && keys2 instanceof Keys.FieldPositionKeys && keys1.areCompatibale(keys2) ) { - + @SuppressWarnings("unchecked") Keys.SelectorFunctionKeys selectorKeys1 = (Keys.SelectorFunctionKeys) keys1; - + int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions(); - - PlanUnwrappingCoGroupOperator po = - translateSelectorFunctionCoGroupLeft(selectorKeys1, logicalKeyPositions2, function, + + PlanUnwrappingCoGroupOperator po = + translateSelectorFunctionCoGroupLeft(selectorKeys1, logicalKeyPositions2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - + // set dop po.setDegreeOfParallelism(this.getParallelism()); - + return po; } else { throw new UnsupportedOperationException("Unrecognized or incompatible key types."); } } - - + + private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroup( - Keys.SelectorFunctionKeys rawKeys1, Keys.SelectorFunctionKeys rawKeys2, - CoGroupFunction function, + Keys.SelectorFunctionKeys rawKeys1, Keys.SelectorFunctionKeys rawKeys2, + CoGroupFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { @@ -166,177 +170,177 @@ private static PlanUnwrappingCoGroupOperator tr final Keys.SelectorFunctionKeys keys1 = (Keys.SelectorFunctionKeys) rawKeys1; @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys2 = (Keys.SelectorFunctionKeys) rawKeys2; - + final TypeInformation> typeInfoWithKey1 = new TupleTypeInfo>(keys1.getKeyType(), inputType1); final TypeInformation> typeInfoWithKey2 = new TupleTypeInfo>(keys2.getKeyType(), inputType2); - + final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - + final PlanMapOperator> keyMapper1 = new PlanMapOperator>(extractor1, "Key Extractor 1", inputType1, typeInfoWithKey1); final PlanMapOperator> keyMapper2 = new PlanMapOperator>(extractor2, "Key Extractor 2", inputType2, typeInfoWithKey2); final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); - + cogroup.setFirstInput(keyMapper1); cogroup.setSecondInput(keyMapper2); - + keyMapper1.setInput(input1); keyMapper2.setInput(input2); // set dop keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism()); keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism()); - + return cogroup; } - + private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroupRight( - int[] logicalKeyPositions1, Keys.SelectorFunctionKeys rawKeys2, - CoGroupFunction function, + int[] logicalKeyPositions1, Keys.SelectorFunctionKeys rawKeys2, + CoGroupFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { if(!inputType1.isTupleType()) { throw new InvalidParameterException("Should not happen."); } - + @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys2 = (Keys.SelectorFunctionKeys) rawKeys2; - + final TypeInformation> typeInfoWithKey1 = new TupleTypeInfo>(keys2.getKeyType(), inputType1); // assume same key, checked by Key.areCompatibale() before final TypeInformation> typeInfoWithKey2 = new TupleTypeInfo>(keys2.getKeyType(), inputType2); - + final TupleKeyExtractingMapper extractor1 = new TupleKeyExtractingMapper(logicalKeyPositions1[0]); final KeyExtractingMapper extractor2 = new KeyExtractingMapper(keys2.getKeyExtractor()); - + final PlanMapOperator> keyMapper1 = new PlanMapOperator>(extractor1, "Key Extractor 1", inputType1, typeInfoWithKey1); final PlanMapOperator> keyMapper2 = new PlanMapOperator>(extractor2, "Key Extractor 2", inputType2, typeInfoWithKey2); - + final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); - + cogroup.setFirstInput(keyMapper1); cogroup.setSecondInput(keyMapper2); - + keyMapper1.setInput(input1); keyMapper2.setInput(input2); // set dop keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism()); keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism()); - + return cogroup; } - + private static PlanUnwrappingCoGroupOperator translateSelectorFunctionCoGroupLeft( Keys.SelectorFunctionKeys rawKeys1, int[] logicalKeyPositions2, - CoGroupFunction function, + CoGroupFunction function, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType, String name, Operator input1, Operator input2) { if(!inputType2.isTupleType()) { throw new InvalidParameterException("Should not happen."); } - + @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys1 = (Keys.SelectorFunctionKeys) rawKeys1; - + final TypeInformation> typeInfoWithKey1 = new TupleTypeInfo>(keys1.getKeyType(), inputType1); // assume same key, checked by Key.areCompatibale() before final TypeInformation> typeInfoWithKey2 = new TupleTypeInfo>(keys1.getKeyType(), inputType2); - + final KeyExtractingMapper extractor1 = new KeyExtractingMapper(keys1.getKeyExtractor()); final TupleKeyExtractingMapper extractor2 = new TupleKeyExtractingMapper(logicalKeyPositions2[0]); - + final PlanMapOperator> keyMapper1 = new PlanMapOperator>(extractor1, "Key Extractor 1", inputType1, typeInfoWithKey1); final PlanMapOperator> keyMapper2 = new PlanMapOperator>(extractor2, "Key Extractor 2", inputType2, typeInfoWithKey2); - + final PlanUnwrappingCoGroupOperator cogroup = new PlanUnwrappingCoGroupOperator(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); - + cogroup.setFirstInput(keyMapper1); cogroup.setSecondInput(keyMapper2); - + keyMapper1.setInput(input1); keyMapper2.setInput(input2); // set dop keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism()); keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism()); - + return cogroup; } // -------------------------------------------------------------------------------------------- // Builder classes for incremental construction // -------------------------------------------------------------------------------------------- - + public static final class CoGroupOperatorSets { - + private final DataSet input1; private final DataSet input2; - + public CoGroupOperatorSets(DataSet input1, DataSet input2) { if (input1 == null || input2 == null) { throw new NullPointerException(); } - + this.input1 = input1; this.input2 = input2; } - + public CoGroupOperatorSetsPredicate where(int... fields) { return new CoGroupOperatorSetsPredicate(new Keys.FieldPositionKeys(fields, input1.getType())); } - + public CoGroupOperatorSetsPredicate where(KeySelector keyExtractor) { return new CoGroupOperatorSetsPredicate(new Keys.SelectorFunctionKeys(keyExtractor, input1.getType())); } - + public CoGroupOperatorSetsPredicate where(String keyExpression) { return new CoGroupOperatorSetsPredicate(new Keys.ExpressionKeys(keyExpression, input1.getType())); } - + // ---------------------------------------------------------------------------------------- - + public final class CoGroupOperatorSetsPredicate { - + private final Keys keys1; - + private CoGroupOperatorSetsPredicate(Keys keys1) { if (keys1 == null) { throw new NullPointerException(); } - + if (keys1.isEmpty()) { throw new InvalidProgramException("The join keys must not be empty."); } - + this.keys1 = keys1; } - - + + public CoGroupOperatorWithoutFunction equalTo(int... fields) { return createCoGroupOperator(new Keys.FieldPositionKeys(fields, input2.getType())); - + } - + public CoGroupOperatorWithoutFunction equalTo(KeySelector keyExtractor) { return createCoGroupOperator(new Keys.SelectorFunctionKeys(keyExtractor, input2.getType())); } - + public CoGroupOperatorWithoutFunction equalTo(String keyExpression) { return createCoGroupOperator(new Keys.ExpressionKeys(keyExpression, input2.getType())); } - - + + private CoGroupOperatorWithoutFunction createCoGroupOperator(Keys keys2) { if (keys2 == null) { throw new NullPointerException(); } - + if (keys2.isEmpty()) { throw new InvalidProgramException("The join keys must not be empty."); } - + if (!keys1.areCompatibale(keys2)) { throw new InvalidProgramException("The pair of join keys are not compatible with each other."); } - + return new CoGroupOperatorWithoutFunction(keys2); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java index 64888f7cd631d..8a48657b560b0 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java @@ -36,9 +36,9 @@ * * @see DataSet */ -public class CrossOperator +public class CrossOperator extends TwoInputUdfOperator> { - + private final CrossFunction function; protected CrossOperator(DataSet input1, DataSet input2, @@ -49,40 +49,44 @@ protected CrossOperator(DataSet input1, DataSet input2, this.function = function; } - + @Override protected Operator translateToDataFlow(Operator input1, Operator input2) { - + String name = getName() != null ? getName() : function.getClass().getName(); // create operator PlanCrossOperator po = new PlanCrossOperator(function, name, getInput1Type(), getInput2Type(), getResultType()); // set inputs po.setFirstInput(input1); po.setSecondInput(input2); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } // set dop po.setDegreeOfParallelism(this.getParallelism()); - + return po; } - + // -------------------------------------------------------------------------------------------- // Builder classes for incremental construction // -------------------------------------------------------------------------------------------- - + public static final class DefaultCross extends CrossOperator> { - + private final DataSet input1; private final DataSet input2; - + public DefaultCross(DataSet input1, DataSet input2) { - super(input1, input2, (CrossFunction>) new DefaultCrossFunction(), + super(input1, input2, (CrossFunction>) new DefaultCrossFunction(), new TupleTypeInfo>(input1.getType(), input2.getType())); - + if (input1 == null || input2 == null) { throw new NullPointerException(); } - + this.input1 = input1; this.input2 = input2; } @@ -172,16 +176,16 @@ public static final class ProjectCrossFunction extends /** * Instantiates and configures a ProjectCrossFunction. * Creates output tuples by copying fields of crossed input tuples (or a full input object) into an output tuple. - * - * @param fields List of indexes fields that should be copied to the output tuple. - * If the full input object should be copied (for example in case of a non-tuple input) the index should be -1. + * + * @param fields List of indexes fields that should be copied to the output tuple. + * If the full input object should be copied (for example in case of a non-tuple input) the index should be -1. * @param isFromFirst List of flags indicating whether the field should be copied from the first (true) or the second (false) input. * @param outTupleInstance An instance of an output tuple. */ private ProjectCrossFunction(int[] fields, boolean[] isFromFirst, R outTupleInstance) { - + if(fields.length != isFromFirst.length) { - throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); + throw new IllegalArgumentException("Fields and isFromFirst arrays must have same length!"); } this.fields = fields; this.isFromFirst = isFromFirst; @@ -223,7 +227,7 @@ public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes this.ds1 = ds1; this.ds2 = ds2; - + boolean isFirstTuple; boolean isSecondTuple; @@ -280,7 +284,7 @@ public CrossProjection(DataSet ds1, DataSet ds2, int[] firstFieldIndexes if(isTuple) { this.isFieldInFirst = new boolean[this.fieldIndexes.length]; - + // check field indexes and adapt to position in tuple int maxFieldIndex = firstInput ? numFieldsDs1 : numFieldsDs2; for(int i=0; i projectSecond(int... secondFieldIndexes) { // GENERATED FROM eu.stratosphere.api.java.tuple.TupleGenerator. /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -455,7 +459,7 @@ public ProjectCross> types(Class type0) { if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -463,13 +467,13 @@ public ProjectCross> types(Class type0) { } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -478,7 +482,7 @@ public ProjectCross> types(Class type0, Clas if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -486,14 +490,14 @@ public ProjectCross> types(Class type0, Clas } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -502,7 +506,7 @@ public ProjectCross> types(Class typ if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -510,15 +514,15 @@ public ProjectCross> types(Class typ } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. * @param type3 The class of field '3' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -527,7 +531,7 @@ public ProjectCross> types(Class if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -535,16 +539,16 @@ public ProjectCross> types(Class } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. * @param type3 The class of field '3' of the result tuples. * @param type4 The class of field '4' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -553,7 +557,7 @@ public ProjectCross> typ if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -561,9 +565,9 @@ public ProjectCross> typ } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. @@ -571,7 +575,7 @@ public ProjectCross> typ * @param type4 The class of field '4' of the result tuples. * @param type5 The class of field '5' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -580,7 +584,7 @@ public ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -588,9 +592,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -616,9 +620,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -645,9 +649,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -675,9 +679,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -706,9 +710,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -738,9 +742,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -771,9 +775,9 @@ public ProjectCross ProjectCross ProjectCross[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -805,9 +809,9 @@ public ProjectCross ProjectCross ProjectCross if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -840,9 +844,9 @@ public ProjectCross } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. @@ -859,7 +863,7 @@ public ProjectCross * @param type13 The class of field '13' of the result tuples. * @param type14 The class of field '14' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -868,7 +872,7 @@ public Project if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -876,9 +880,9 @@ public Project } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. @@ -896,7 +900,7 @@ public Project * @param type14 The class of field '14' of the result tuples. * @param type15 The class of field '15' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -905,7 +909,7 @@ public Pr if(types.length != this.fieldIndexes.length) { throw new IllegalArgumentException("Numbers of projected fields and types do not match."); } - + TypeInformation[] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -913,9 +917,9 @@ public Pr } /** - * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. - * Requires the classes of the fields of the resulting tuples. - * + * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. + * Requires the classes of the fields of the resulting tuples. + * * @param type0 The class of field '0' of the result tuples. * @param type1 The class of field '1' of the result tuples. * @param type2 The class of field '2' of the result tuples. @@ -934,7 +938,7 @@ public Pr * @param type15 The class of field '15' of the result tuples. * @param type16 The class of field '16' of the result tuples. * @return The projected data set. - * + * * @see Tuple * @see DataSet */ @@ -943,7 +947,7 @@ public [] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -951,9 +955,9 @@ public [] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -990,9 +994,9 @@ public [] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -1030,9 +1034,9 @@ public [] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -1071,9 +1075,9 @@ public [] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -1113,9 +1117,9 @@ public [] fTypes = extractFieldTypes(fieldIndexes, types); TupleTypeInfo> tType = new TupleTypeInfo>(fTypes); @@ -1324,11 +1328,11 @@ private TypeInformation[] extractFieldTypes(int[] fields, Class[] givenTyp return fieldTypes; } } - + // -------------------------------------------------------------------------------------------- // default join functions // -------------------------------------------------------------------------------------------- - + public static final class DefaultCrossFunction extends CrossFunction> { private static final long serialVersionUID = 1L; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java index 5fb119f2cb650..5e77651cbe6b0 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java @@ -26,28 +26,32 @@ * @param The type of the data set created by the operator. */ public class FlatMapOperator extends SingleInputUdfOperator> { - + protected final FlatMapFunction function; - - + + public FlatMapOperator(DataSet input, FlatMapFunction function) { super(input, TypeExtractor.getFlatMapReturnTypes(function, input.getType())); - + if (function == null) { throw new NullPointerException("FlatMap function must not be null."); } - + this.function = function; } @Override protected Operator translateToDataFlow(Operator input) { - + String name = getName() != null ? getName() : function.getClass().getName(); // create operator PlanFlatMapOperator po = new PlanFlatMapOperator(function, name, getInputType(), getResultType()); // set input po.setInput(input); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } // set dop if(this.getParallelism() > 0) { // use specified dop @@ -56,7 +60,7 @@ protected Operator translateToDataFlow(Operator input) { // if no dop has been specified, use dop of input operator to enable chaining po.setDegreeOfParallelism(input.getDegreeOfParallelism()); } - + return po; } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java index 21bba8b1f9357..b8c95720da1dd 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java @@ -217,6 +217,10 @@ else if (super.keys1 instanceof Keys.FieldPositionKeys // set inputs po.setFirstInput(input1); po.setSecondInput(input2); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } // set dop po.setDegreeOfParallelism(this.getParallelism()); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java index bd797dd769751..ace3e73ec5d0f 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java @@ -26,28 +26,32 @@ * @param The type of the data set created by the operator. */ public class MapOperator extends SingleInputUdfOperator> { - + protected final MapFunction function; - - + + public MapOperator(DataSet input, MapFunction function) { super(input, TypeExtractor.getMapReturnTypes(function, input.getType())); - + if (function == null) { throw new NullPointerException("Map function must not be null."); } - + this.function = function; } @Override protected Operator translateToDataFlow(Operator input) { - + String name = getName() != null ? getName() : function.getClass().getName(); // create operator PlanMapOperator po = new PlanMapOperator(function, name, getInputType(), getResultType()); // set input po.setInput(input); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } // set dop if(this.getParallelism() > 0) { // use specified dop @@ -56,8 +60,8 @@ protected Operator translateToDataFlow(Operator input) { // if no dop has been specified, use dop of input operator to enable chaining po.setDegreeOfParallelism(input.getDegreeOfParallelism()); } - + return po; } - + } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java index 392badc338092..359f1e4daad1a 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java @@ -144,6 +144,10 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { // set input po.setInput(input); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } // set dop po.setDegreeOfParallelism(this.getParallelism()); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java index 25598d904b810..e9afe13bbc7ef 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java @@ -74,6 +74,11 @@ protected Operator translateToDataFlow(Operator input) { if (grouper == null) { // non grouped reduce PlanReduceOperator po = new PlanReduceOperator(function, new int[0], name, getInputType()); + //set semantic properties + if (this.getSematicProperties() != null) { + po.setSemanticProperties(this.getSematicProperties()); + } + // set input po.setInput(input); // the degree of parallelism for a non grouped reduce can only be 1 diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java index cb87c974a63b5..62e0eae90d21c 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java @@ -20,15 +20,16 @@ import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; import eu.stratosphere.configuration.Configuration; /** - * The SingleInputUdfOperator is the base class of all unary operators that execute + * The SingleInputUdfOperator is the base class of all unary operators that execute * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that * have one input (such as {@link MapFunction} or {@link ReduceFunction}). *

          - * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization + * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization * through configuration objects, and semantic properties. * @param The data type of the input data set. * @param The data type of the returned data set. @@ -37,77 +38,83 @@ public abstract class SingleInputUdfOperator implements UdfOperator { private Configuration parameters; - + private Map> broadcastVariables; - + private SingleInputSemanticProperties udfSemantics; - + // -------------------------------------------------------------------------------------------- - + /** * Creates a new operators with the given data set as input. The given result type - * describes the data type of the elements in the data set produced by the operator. - * + * describes the data type of the elements in the data set produced by the operator. + * * @param input The data set that is the input to the operator. * @param resultType The type of the elements in the resulting data set. */ protected SingleInputUdfOperator(DataSet input, TypeInformation resultType) { super(input, resultType); } - + // -------------------------------------------------------------------------------------------- // Fluent API methods // -------------------------------------------------------------------------------------------- - + @Override public O withParameters(Configuration parameters) { this.parameters = parameters; - + @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; } - + @Override public O withBroadcastSet(DataSet data, String name) { if (this.broadcastVariables == null) { this.broadcastVariables = new HashMap>(); } - + this.broadcastVariables.put(name, data); - + @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; } - - + + public O withProperties(String[] constantSet, String constantSetExcept, String readSet) { + SingleInputSemanticProperties props = SemanticPropUtil.getSemanticPropsSingleFromString(constantSet, constantSetExcept, readSet, this.getInputType(), this.getResultType()); + this.setSemanticProperties(props); + O returnType = (O) this; + return returnType; + } + // -------------------------------------------------------------------------------------------- // Accessors // -------------------------------------------------------------------------------------------- - + @Override public Map> getBroadcastSets() { return this.broadcastVariables == null ? Collections.>emptyMap() : Collections.unmodifiableMap(this.broadcastVariables); } - + @Override public Configuration getParameters() { return this.parameters; } - + @Override public SingleInputSemanticProperties getSematicProperties() { return this.udfSemantics; } - + /** * Sets the semantic properties for the user-defined function (UDF). The semantic properties * define how fields of tuples and other objects are modified or preserved through this UDF. * The configured properties can be retrieved via {@link UdfOperator#getSematicProperties()}. - * + * * @param properties The semantic properties for the UDF. * @see UdfOperator#getSematicProperties() */ diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java index e4be8e970e6b5..fb4131b6b28e1 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java @@ -20,15 +20,16 @@ import eu.stratosphere.api.common.operators.DualInputSemanticProperties; import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; import eu.stratosphere.configuration.Configuration; /** - * The TwoInputUdfOperator is the base class of all binary operators that execute + * The TwoInputUdfOperator is the base class of all binary operators that execute * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that * have two inputs (such as {@link JoinFunction} or {@link CoGroupFunction}). *

          - * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization + * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization * through configuration objects, and semantic properties. * * @param The data type of the first input data set. @@ -39,17 +40,17 @@ public abstract class TwoInputUdfOperator implements UdfOperator { private Configuration parameters; - + private Map> broadcastVariables; - + private DualInputSemanticProperties udfSemantics; - + // -------------------------------------------------------------------------------------------- - + /** * Creates a new operators with the two given data sets as inputs. The given result type - * describes the data type of the elements in the data set produced by the operator. - * + * describes the data type of the elements in the data set produced by the operator. + * * @param input1 The data set for the first input. * @param input2 The data set for the second input. * @param resultType The type of the elements in the resulting data set. @@ -57,59 +58,71 @@ public abstract class TwoInputUdfOperator input1, DataSet input2, TypeInformation resultType) { super(input1, input2, resultType); } - + // -------------------------------------------------------------------------------------------- // Fluent API methods // -------------------------------------------------------------------------------------------- - + @Override public O withParameters(Configuration parameters) { this.parameters = parameters; - + @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; } - + @Override public O withBroadcastSet(DataSet data, String name) { if (this.broadcastVariables == null) { this.broadcastVariables = new HashMap>(); } - + this.broadcastVariables.put(name, data); - + @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; } - + + /* + Allows to give specifications about constant sets directly in the code. Null values are allowed for not specified sets. + */ + public O withProperties(String[] constantSetFirst, String[] constantSetSecond, String constExceptFirst, String constExceptSecond, String readSetFirst, String readSetSecond) { + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(constantSetFirst, constantSetSecond, + constExceptFirst, constExceptSecond, readSetFirst, readSetSecond, this.getInput1Type(), this.getInput2Type(), this.getResultType()); + this.setSemanticProperties(dsp); + + O returnType = (O) this; + return returnType; + } + // -------------------------------------------------------------------------------------------- // Accessors // -------------------------------------------------------------------------------------------- - + @Override public Map> getBroadcastSets() { return this.broadcastVariables == null ? Collections.>emptyMap() : Collections.unmodifiableMap(this.broadcastVariables); } - + @Override public Configuration getParameters() { return this.parameters; } - + @Override public DualInputSemanticProperties getSematicProperties() { return this.udfSemantics; } - + /** * Sets the semantic properties for the user-defined function (UDF). The semantic properties * define how fields of tuples and other objects are modified or preserved through this UDF. * The configured properties can be retrieved via {@link UdfOperator#getSematicProperties()}. - * + * * @param properties The semantic properties for the UDF. * @see UdfOperator#getSematicProperties() */ diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java index 687d08d61cde6..2f83063640414 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java @@ -43,6 +43,42 @@ public void testSimpleCase() { Assert.assertTrue(fs.contains(3)); } + @Test + public void testSimpleCaseWildCard() { + String[] constantFields = {"*"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(1)); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(2)); + + fs = sp.getForwardedField(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(0)); + } + + @Test + public void testSimpleCaseWildCard2() { + String[] constantFields = {"1->*"}; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 3); + Assert.assertTrue(fs.contains(0)); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + Assert.assertTrue(sp.getForwardedField(0) == null); + Assert.assertTrue(sp.getForwardedField(2) == null); + } + @Test public void testConstantFieldsExcept() { String constantFieldsExcept = "1"; From 37b4fd33e4e533b9b56c5dfd4b70992610a7bea3 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Thu, 15 May 2014 18:08:05 +0200 Subject: [PATCH 037/182] Filter UDFs always considers all fields as constants --- .../main/java/eu/stratosphere/compiler/dag/FilterNode.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/FilterNode.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/FilterNode.java index 7538e25cef85f..589e211b6dbdd 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/FilterNode.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/dag/FilterNode.java @@ -40,6 +40,11 @@ public FilterOperatorBase getPactContract() { public String getName() { return "Filter"; } + + @Override + public boolean isFieldConstant(int input, int fieldNumber) { + return true; + } @Override protected List getPossibleProperties() { From ac7b215d60ed852be3fb6d59dcfe6ead11812703 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Thu, 15 May 2014 18:13:57 +0200 Subject: [PATCH 038/182] Fix group reduce compilation test --- .../stratosphere/pact/compiler/GroupReduceCompilationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupReduceCompilationTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupReduceCompilationTest.java index 68f0dbfb443f2..7de3fbc3b45e4 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupReduceCompilationTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupReduceCompilationTest.java @@ -116,7 +116,7 @@ public void reduce(Iterator values, Collector out) {} // check that both reduce and combiner have the same strategy assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.ALL_GROUP_REDUCE, combineNode.getDriverStrategy()); + assertEquals(DriverStrategy.ALL_GROUP_COMBINE, combineNode.getDriverStrategy()); // check DOP assertEquals(8, sourceNode.getDegreeOfParallelism()); From c426650da610f9a8ebbef0ce6e67539058215c4f Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Thu, 15 May 2014 21:11:48 +0200 Subject: [PATCH 039/182] Merged and adjusted semantic annotations --- .../spargel/java/VertexCentricIteration.java | 6 +- .../java/graph/ConnectedComponents.java | 6 + .../api/java/IterativeDataSet.java | 2 +- .../java/functions/FunctionAnnotation.java | 368 ++++++++++-------- .../api/java/functions/SemanticPropUtil.java | 100 ++--- .../api/java/operators/AggregateOperator.java | 2 +- .../api/java/operators/CoGroupOperator.java | 7 +- .../api/java/operators/CrossOperator.java | 7 +- .../api/java/operators/DistinctOperator.java | 3 +- .../api/java/operators/FilterOperator.java | 3 +- .../api/java/operators/FlatMapOperator.java | 7 +- .../api/java/operators/JoinOperator.java | 15 +- .../api/java/operators/MapOperator.java | 11 +- .../java/operators/OperatorTranslation.java | 43 +- .../api/java/operators/ProjectOperator.java | 2 +- .../java/operators/ReduceGroupOperator.java | 6 +- .../api/java/operators/ReduceOperator.java | 6 +- .../java/operators/SingleInputOperator.java | 2 +- .../operators/SingleInputUdfOperator.java | 5 +- .../api/java/operators/TwoInputOperator.java | 2 +- .../java/operators/TwoInputUdfOperator.java | 21 +- .../translation/PlanReduceOperator.java | 2 + .../java/functions/SemanticPropUtilTest.java | 198 ++++++++++ .../nephele/taskmanager/TaskManager.java | 6 + .../semanticprops/SemanticPropUtilTest.java | 187 --------- 25 files changed, 549 insertions(+), 468 deletions(-) create mode 100644 stratosphere-java/src/test/java/eu/stratosphere/api/java/functions/SemanticPropUtilTest.java delete mode 100644 stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java diff --git a/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/VertexCentricIteration.java b/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/VertexCentricIteration.java index 97a628d01e86d..85841eb31ba56 100644 --- a/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/VertexCentricIteration.java +++ b/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/VertexCentricIteration.java @@ -19,6 +19,7 @@ import org.apache.commons.lang3.Validate; import eu.stratosphere.api.common.aggregators.Aggregator; +import eu.stratosphere.api.common.operators.DualInputOperator; import eu.stratosphere.api.common.operators.DualInputSemanticProperties; import eu.stratosphere.api.common.operators.Operator; import eu.stratosphere.api.java.DataSet; @@ -456,7 +457,7 @@ private GraphIterationOperator(DataSet> initialVe } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected DualInputOperator translateToDataFlow(Operator input1, Operator input2) { final String name = (getName() != null) ? getName() : "Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")"; @@ -486,7 +487,7 @@ protected Operator translateToDataFlow(Operator input1, Operator input2) { updater.setFirstInput(messenger); updater.setSecondInput(iteration.getSolutionSet()); - // let the opertor know that we preserve the key field + // let the operator know that we preserve the key field DualInputSemanticProperties semanticProps = new DualInputSemanticProperties(); semanticProps.addForwardedField1(0, 0); semanticProps.addForwardedField2(0, 0); @@ -501,7 +502,6 @@ protected Operator translateToDataFlow(Operator input1, Operator input2) { messenger.setFirstInput(input2); return iteration; - } } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java index a2f402a922b17..6f6b9894baf7c 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java @@ -19,6 +19,9 @@ import eu.stratosphere.api.java.ExecutionEnvironment; import eu.stratosphere.api.java.aggregation.Aggregations; import eu.stratosphere.api.java.functions.FlatMapFunction; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFields; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsFirst; +import eu.stratosphere.api.java.functions.FunctionAnnotation.ConstantFieldsSecond; import eu.stratosphere.api.java.functions.JoinFunction; import eu.stratosphere.api.java.functions.MapFunction; import eu.stratosphere.api.java.tuple.Tuple1; @@ -108,6 +111,7 @@ public static void main(String... args) throws Exception { /** * Function that turns a value into a 2-tuple where both fields are that value. */ + @ConstantFields("0 -> 0,1") public static final class DuplicateValue extends MapFunction> { @Override @@ -121,6 +125,8 @@ public Tuple2 map(T value) { * a vertex is associated with, with a (Source-Vertex-ID, Target-VertexID) edge. The function * produces a (Target-vertex-ID, Component-ID) pair. */ + @ConstantFieldsFirst("1 -> 0") + @ConstantFieldsSecond("1 -> 1") public static final class NeighborWithComponentIDJoin extends JoinFunction, Tuple2, Tuple2> { @Override diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/IterativeDataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/IterativeDataSet.java index facfaa7391b54..b7adacab88146 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/IterativeDataSet.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/IterativeDataSet.java @@ -135,7 +135,7 @@ public AggregatorRegistry getAggregators() { // -------------------------------------------------------------------------------------------- @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { // All the translation magic happens when the iteration end is encountered. throw new UnsupportedOperationException("This should never happen."); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java index 45b1b73976012..0e63d68fd4246 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/FunctionAnnotation.java @@ -27,22 +27,39 @@ /** * This class defines the semantic assertions that can be added to functions. * The assertions are realized as java annotations, to be added to the class declaration of - * the class that realized the user function. For example, to declare the ConstantFields - * annotation for a map-type function that simply copies some fields, - * use it the following way: + * the class that implements the functions. For example, to declare the ConstantFields + * annotation for a map-type function that simply copies some fields, use it the following way: * - *

          - * \@ConstantFields({"0->0,1", "1->2"}) + *
          + * {@code
          + * @ConstantFields({"0->0,1", "1->2"})
            * public class MyMapper extends FlatMapFunction, Tuple3>
            * {
            *     public void flatMap(Tuple3 value, Collector> out) {
          -			value.f2 = value.f1
          -            value.f1 = value.f0;
          -			out.collect(value);
          -		}
          + *         value.f2 = value.f1
          + *         value.f1 = value.f0;
          + *         out.collect(value);
          + *     }
            * }
          - * 
          - * + * } + *
+ *

+ * All annotations takes String arrays. The Strings represent the source and destination fields. + * The transition is represented by the arrow "->". + * Fields are described by their tuple position (and later also the names of the fields in the objects). + * The left hand side of the arrow always describes the fields in the input value(s), i.e. the value that + * is passed as a parameter to the function call, or the values obtained from the input iterator. The right + * hand side of the arrow describes the field in the value returned from the function. If the right hand side + * is omitted, the a field is assumed to stay exactly the same, i.e. the field itself is unmodified, rather + * than that the value is placed into another field. + *

+ * + * It is very important to follow a conservative strategy when specifying constant fields. + * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be + * declared as such! Otherwise, the correct execution of a program can not be guaranteed. So if in doubt, + * do not add a field to this set. + * + *

* Be aware that some annotations should only be used for functions with as single input * ({@link MapFunction}, {@link ReduceFunction}) and some only for stubs with two inputs * ({@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}). @@ -50,28 +67,31 @@ public class FunctionAnnotation { /** - * Specifies the fields of an input tuple or custom object that are unchanged in the output of - * a stub with a single input ( {@link MapFunction}, {@link ReduceFunction}). - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * - * The annotation takes one String array. The Strings represent the source and destination fields - * of the constant fields. The transition is represented by the string "->". The following would be a - * valid annotation "1->2,3". - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * + * This annotation declares that a function leaves certain fields of its input values unmodified and + * only "forwards" or "copies" them to the return value. The annotation is applicable to unary + * functions, like for example {@link MapFunction}, {@link ReduceFunction}, or {@link FlatMapFunction}. + *

+ * The following example illustrates a function that keeps the tuple's field zero constant: + *

+	 * {@code
+	 * @ConstantFields("0")
+	 * public class MyMapper extends MapFunction, Tuple2>
+	 * {
+	 *     public Tuple2 map(Tuple3 value) {
+	 *         return new Tuple2(value.f0, value.f1 * 0.5);
+	 *     }
+	 * }
+	 * }
+	 * 
+ *

+ * (Note that you could equivalently write {@code @ConstantFields("0 -> 0")}. + *

* This annotation is mutually exclusive with the {@link ConstantFieldsExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsExcept} annotation is not set, it is - * assumed that no field is constant. - * + *

+ * If neither this annotation, nor the {@link ConstantFieldsExcept} annotation are set, it is + * assumed that no field in the input is forwarded/copied unmodified. + * + * @see ConstantFieldsExcept */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) @@ -80,29 +100,32 @@ public class FunctionAnnotation { } /** - * Specifies the fields of an input tuple or custom object of the first input that are unchanged in - * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * - * The annotation takes one String array. The Strings represent the source and destination fields - * of the constant fields. The transition is represented by the string "->". The following would be a - * valid annotation "1->2,3". - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * + * This annotation declares that a function leaves certain fields of its first input values unmodified and + * only "forwards" or "copies" them to the return value. The annotation is applicable to binary + * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + *

+ * The following example illustrates a join function that copies fields from the first and second input to the + * return value: + *

+	 * {@code
+	 * @ConstantFieldsFirst("1 -> 0")
+	 * @ConstantFieldsFirst("1 -> 1")
+	 * public class MyJoin extends JoinFunction, Tuple2, Tuple2>
+	 * {
+	 *     public Tuple2 map(Tuple2 first, Tuple2 second) {
+	 *         return new Tuple2(first.f1, second.f1);
+	 *     }
+	 * }
+	 * }
+	 * 
+ *

* This annotation is mutually exclusive with the {@link ConstantFieldsFirstExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsFirstExcept} annotation is not set, it is - * assumed that no field is constant. - * - * + *

+ * If neither this annotation, nor the {@link ConstantFieldsFirstExcept} annotation are set, it is + * assumed that no field in the first input is forwarded/copied unmodified. + * + * @see ConstantFieldsSecond + * @see ConstantFields */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) @@ -111,28 +134,32 @@ public class FunctionAnnotation { } /** - * Specifies the fields of an input tuple or custom object of the second input that are unchanged in - * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * - * The annotation takes one String array. The Strings represent the source and destination fields - * of the constant fields. The transition is represented by the string "->". The following would be a - * valid annotation "1->2,3". - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * - * This annotation is mutually exclusive with the {@link ConstantFieldsSecondExcept} annotation. - * - * If this annotation and the {@link ConstantFieldsSecondExcept} annotation is not set, it is - * assumed that no field is constant. - * + * This annotation declares that a function leaves certain fields of its second input values unmodified and + * only "forwards" or "copies" them to the return value. The annotation is applicable to binary + * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + *

+ * The following example illustrates a join function that copies fields from the first and second input to the + * return value: + *

+	 * {@code
+	 * @ConstantFieldsFirst("1 -> 0")
+	 * @ConstantFieldsFirst("1 -> 1")
+	 * public class MyJoin extends JoinFunction, Tuple2, Tuple2>
+	 * {
+	 *     public Tuple2 map(Tuple2 first, Tuple2 second) {
+	 *         return new Tuple2(first.f1, second.f1);
+	 *     }
+	 * }
+	 * }
+	 * 
+ *

+ * This annotation is mutually exclusive with the {@link ConstantFieldsSecond} annotation. + *

+ * If neither this annotation, nor the {@link ConstantFieldsSecondExcept} annotation are set, it is + * assumed that no field in the second input is forwarded/copied unmodified. + * + * @see ConstantFieldsFirst + * @see ConstantFields */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) @@ -141,131 +168,146 @@ public class FunctionAnnotation { } /** - * Specifies the fields of an input tuple or custom object that are changed in the output of - * a stub with a single input ( {@link MapFunction}, {@link ReduceFunction}). All other - * fields are assumed to be constant. - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * + * This annotation declares that a function changes certain fields of its input values, while leaving all + * others unmodified and in place in the return value. The annotation is applicable to unary + * functions, like for example {@link MapFunction}, {@link ReduceFunction}, or {@link FlatMapFunction}. + *

+ * The following example illustrates that at the example of a Map function. + *

+	 * {@code
+	 * @ConstantFieldsExcept("1")
+	 * public class MyMapper extends MapFunction, Tuple3>
+	 * {
+	 *     public Tuple3 map(Tuple3 value) {
+	 *         return new Tuple3(value.f0, value.f2 / 2, value.f2);
+	 *     }
+	 * }
+	 * }
+	 * 
+ *

* The annotation takes one String array specifying the positions of the input types that do not remain constant. - * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model - * more complex situations use the \@ConstantFields annotation. - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * + * When this annotation is used, it is assumed that all other values remain at the same position in input and output. + * To model more complex situations use the {@link @ConstantFields}s annotation. + *

* This annotation is mutually exclusive with the {@link ConstantFields} annotation. - * - * If this annotation and the {@link ConstantFields} annotation is not set, it is - * assumed that no field is constant. - * + *

+ * If neither this annotation, nor the {@link ConstantFields} annotation are set, it is + * assumed that no field in the input is forwarded/copied unmodified. + * + * @see ConstantFieldsExcept */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsExcept { - String value(); + String[] value(); } /** - * Specifies the fields of an input tuple or custom object of the first input that are changed in - * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) - * All other fields are assumed to be constant. - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * - * The annotation takes one String array specifying the positions of the input types that do not remain constant. - * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model - * more complex situations use the \@ConstantFields annotation. - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * - * This annotation is mutually exclusive with the {@link ConstantFieldsFirst} annotation. - * - * If this annotation and the {@link ConstantFieldsFirst} annotation is not set, it is - * assumed that no field is constant. - * + * This annotation declares that a function changes certain fields of its first input value, while leaving all + * others unmodified and in place in the return value. The annotation is applicable to binary + * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + *

+ * The following example illustrates a join function that copies fields from the first and second input to the + * return value: + *

+	 * {@code
+	 * @ConstantFieldsFirstExcept("1")
+	 * public class MyJoin extends JoinFunction, Tuple2, Tuple3>
+	 * {
+	 *     public Tuple3 map(Tuple3 first, Tuple2 second) {
+	 *         return Tuple3(first.f0, second.f1, first.f2);
+	 *     }
+	 * }
+	 * }
+	 * 
+ *

+ * The annotation takes one String array specifying the positions of the input types that do not remain constant. + * When this annotation is used, it is assumed that all other values remain at the same position in input and output. + * To model more complex situations use the {@link @ConstantFields}s annotation. + *

+ * This annotation is mutually exclusive with the {@link ConstantFieldsFirst} + *

+ * If neither this annotation, nor the {@link ConstantFieldsFirst} annotation are set, it is + * assumed that no field in the first input is forwarded/copied unmodified. + * + * @see ConstantFieldsFirst + * @see ConstantFieldsSecond + * @see ConstantFieldsSecondExcept */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsFirstExcept { - String value(); + String[] value(); } - /** - * Specifies the fields of an input tuple or custom object of the second input that are changed in - * the output of a stub with two inputs ( {@link CrossFunction}, {@link JoinFunction}, {@link CoGroupFunction}) - * All other fields are assumed to be constant. - * - * A field is considered to be constant if its value is not changed and copied to the same position of - * output record. - * - * The annotation takes one String array specifying the positions of the input types that do not remain constant. - * When this annotation is used, it is assumed that all other values remain at the same position in input and output. To model - * more complex situations use the \@ConstantFields annotation. - * - * - * It is very important to follow a conservative strategy when specifying constant fields. - * Only fields that are always constant (regardless of value, stub call, etc.) to the output may be - * inserted! Otherwise, the correct execution of a program can not be guaranteed. - * So if in doubt, do not add a field to this set. - * - * - * This annotation is mutually exclusive with the {@link ConstantFieldsSecond} annotation. - * - * If this annotation and the {@link ConstantFieldsSecond} annotation is not set, it is - * assumed that no field is constant. - * + * This annotation declares that a function changes certain fields of its second input value, while leaving all + * others unmodified and in place in the return value. The annotation is applicable to binary + * functions, like for example {@link JoinFunction}, {@link CoGroupFunction}, or {@link CrossFunction}. + *

+ * The following example illustrates a join function that copies fields from the first and second input to the + * return value: + *

+	 * {@code
+	 * @ConstantFieldsSecondExcept("1")
+	 * public class MyJoin extends JoinFunction, Tuple3, Tuple3>
+	 * {
+	 *     public Tuple3 map(Tuple2 first, Tuple3 second) {
+	 *         return Tuple3(second.f0, first.f1, second.f2);
+	 *     }
+	 * }
+	 * }
+	 * 
+ *

+ * The annotation takes one String array specifying the positions of the input types that do not remain constant. + * When this annotation is used, it is assumed that all other values remain at the same position in input and output. + * To model more complex situations use the {@link @ConstantFields}s annotation. + *

+ * This annotation is mutually exclusive with the {@link ConstantFieldsSecond} + *

+ * If neither this annotation, nor the {@link ConstantFieldsSecond} annotation are set, it is + * assumed that no field in the second input is forwarded/copied unmodified. + * + * @see ConstantFieldsFirst + * @see ConstantFieldsFirstExcept + * @see ConstantFieldsSecond */ @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ConstantFieldsSecondExcept { - String value(); + String[] value(); } /** - * Specifies the fields of an input tuple that are accessed in the function. This annotation should be used - * with user defined functions with one input. + * Specifies the fields of the input value of a user-defined that are accessed in the code. + * This annotation can only be used with user-defined functions with one input (Map, Reduce, ...). */ - @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFields { - String value(); + String[] value(); } /** - * Specifies the fields of an input tuple that are accessed in the function. This annotation should be used - * with user defined functions with two inputs. + * Specifies the fields of the first input value of a user-defined that are accessed in the code. + * This annotation can only be used with user-defined functions with two inputs (Join, Cross, ...). */ - @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsSecond { - String value(); + String[] value(); } /** - * Specifies the fields of an input tuple that are accessed in the function. This annotation should be used - * with user defined functions with two inputs. + * Specifies the fields of the second input value of a user-defined that are accessed in the code. + * This annotation can only be used with user-defined functions with two inputs (Join, Cross, ...). */ - @Target(ElementType.TYPE) @Retention(RetentionPolicy.RUNTIME) public @interface ReadFieldsFirst { - String value(); + String[] value(); } + /** * Private constructor to prevent instantiation. This class is intended only as a container. */ @@ -277,10 +319,10 @@ private FunctionAnnotation() {} /** * Reads the annotations of a user defined function with one input and returns semantic properties according to the constant fields annotated. + * * @param udf The user defined function. * @return The DualInputSemanticProperties containing the constant fields. */ - public static Set readSingleConstantAnnotations(UserCodeWrapper udf) { ConstantFields constantSet = udf.getUserCodeAnnotation(ConstantFields.class); ConstantFieldsExcept notConstantSet = udf.getUserCodeAnnotation(ConstantFieldsExcept.class); @@ -381,9 +423,5 @@ public static Set readDualConstantAnnotations(UserCodeWrapper udf return result; } - - - - - } +} diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java index b8d6de76403e4..1eb0ec061fbf6 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java @@ -36,7 +36,11 @@ public class SemanticPropUtil { - private final static String REGEX_ANNOTATION = "\\s*(\\d+)\\s*->((\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))|(\\*))"; + private static final Pattern PATTERN = Pattern.compile("\\s*(\\d+)\\s*->((\\s*(\\d+\\s*,\\s*)*(\\d+\\s*))|(\\*))"); + + private static final Pattern FIELD_SET_CHECK_PATTERN = Pattern.compile("\\s*(\\d+\\s*,\\s*)*(\\d+\\s*)"); + + private static final Pattern DIGIT_PATTERN = Pattern.compile("\\d+"); public static SingleInputSemanticProperties getSemanticPropsSingle(Set set, TypeInformation inType, TypeInformation outType) { if (set == null) { @@ -98,12 +102,11 @@ private static void readConstantSet(SemanticProperties sp, String s, TypeInforma return; } - Pattern check = Pattern.compile(REGEX_ANNOTATION); - Matcher matcher = check.matcher(s); + Matcher matcher = PATTERN.matcher(s); int sourceField = 0; if (!matcher.matches()) { - throw new RuntimeException("Wrong annotation String format. Please read the documentation."); + throw new RuntimeException("Unrecognized annotation string format."); } sourceField = Integer.valueOf(matcher.group(1)); @@ -146,7 +149,6 @@ private static void parseConstantFieldsFirst(String[] cff, DualInputSemanticProp return; } - Pattern check = Pattern.compile(REGEX_ANNOTATION); for (String s : cff) { readConstantSet(dm, s, inType, outType, 0); } @@ -156,91 +158,95 @@ private static void parseConstantFieldsSecond(String[] cfs, DualInputSemanticPro if (cfs == null) { return; } - Pattern check = Pattern.compile(REGEX_ANNOTATION); + for (String s : cfs) { readConstantSet(dm, s, inType, outType, 1); } } - private static void parseConstantFieldsFirstExcept(String cffe, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + private static void parseConstantFieldsFirstExcept(String[] cffe, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { if (cffe == null) { return; } - FieldSet fs = readFieldSetFromString(cffe, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - dm.addForwardedField1(i, i); + for (String str : cffe) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + dm.addForwardedField1(i, i); + } } } } - private static void parseConstantFieldsSecondExcept(String cfse, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + private static void parseConstantFieldsSecondExcept(String[] cfse, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { if (cfse == null) { return; } - FieldSet fs = readFieldSetFromString(cfse, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - dm.addForwardedField2(i, i); + for (String str : cfse) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + dm.addForwardedField2(i, i); + } } } } - private static void parseReadFieldsFirst(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + private static void parseReadFieldsFirst(String[] rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { if (rf == null) { return; } - FieldSet fs = readFieldSetFromString(rf, inType, outType); - dm.addReadFields1(fs); + for (String str : rf) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + dm.addReadFields1(fs); + } } - private static void parseReadFieldsSecond(String rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { + private static void parseReadFieldsSecond(String[] rf, DualInputSemanticProperties dm, TypeInformation inType, TypeInformation outType) { if (rf == null) { return; } - FieldSet fs = readFieldSetFromString(rf, inType, outType); - dm.addReadFields2(fs); + for (String str : rf) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + dm.addReadFields2(fs); + } } private static boolean isValidField(TypeInformation type, int field) { - if (field > type.getArity() || field < 0) { - return false; - } - return true; + return field >= 0 && field < type.getArity(); } - private static void parseConstantFieldsExcept(String cfe, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { + private static void parseConstantFieldsExcept(String[] cfe, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { if (cfe == null) { return; } - FieldSet fs = readFieldSetFromString(cfe, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - sm.addForwardedField(i, i); + for (String str : cfe) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + sm.addForwardedField(i, i); + } } } } private static FieldSet readFieldSetFromString(String s, TypeInformation inType, TypeInformation outType) { - Pattern check = Pattern.compile("\\s*(\\d+\\s*,\\s*)*(\\d+\\s*)"); - Pattern digit = Pattern.compile("\\d+"); - - Matcher matcher = check.matcher(s); + Matcher matcher = FIELD_SET_CHECK_PATTERN.matcher(s); if (!matcher.matches()) { - throw new RuntimeException("Wrong annotation String format. Please read the documentation."); + throw new RuntimeException("Unrecognized annotation string format."); } - matcher = digit.matcher(s); + matcher = DIGIT_PATTERN.matcher(s); FieldSet fs = new FieldSet(); while (matcher.find()) { @@ -253,16 +259,18 @@ private static FieldSet readFieldSetFromString(String s, TypeInformation inTy return fs; } - private static void parseReadFields(String rf, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { + private static void parseReadFields(String[] rf, SingleInputSemanticProperties sm, TypeInformation inType, TypeInformation outType) { if (rf == null) { return; } - FieldSet fs = readFieldSetFromString(rf, inType, outType); - sm.addReadFields(fs); + for (String str : rf) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + sm.addReadFields(fs); + } } - public static SingleInputSemanticProperties getSemanticPropsSingleFromString(String[] constantSet, String constantSetExcept, String readSet, TypeInformation inType, TypeInformation outType) { + public static SingleInputSemanticProperties getSemanticPropsSingleFromString(String[] constantSet, String[] constantSetExcept, String[] readSet, TypeInformation inType, TypeInformation outType) { SingleInputSemanticProperties result = new SingleInputSemanticProperties(); parseConstantFields(constantSet, result, inType, outType); parseConstantFieldsExcept(constantSetExcept, result, inType, outType); @@ -270,8 +278,8 @@ public static SingleInputSemanticProperties getSemanticPropsSingleFromString(Str return result; } - public static DualInputSemanticProperties getSemanticPropsDualFromString(String[] constantSetFirst, String[] constantSetSecond, String constantSetFirstExcept, - String constantSetSecondExcept, String readFieldsFirst, String readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { + public static DualInputSemanticProperties getSemanticPropsDualFromString(String[] constantSetFirst, String[] constantSetSecond, String[] constantSetFirstExcept, + String[] constantSetSecondExcept, String[] readFieldsFirst, String[] readFieldsSecond, TypeInformation inType1, TypeInformation inType2, TypeInformation outType) { DualInputSemanticProperties result = new DualInputSemanticProperties(); parseConstantFieldsFirst(constantSetFirst, result, inType1, outType); parseConstantFieldsSecond(constantSetSecond, result, inType2, outType); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/AggregateOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/AggregateOperator.java index b35f2480dd9e3..3b2d67b21ba3b 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/AggregateOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/AggregateOperator.java @@ -126,7 +126,7 @@ public AggregateOperator and(Aggregations function, int field) { @SuppressWarnings("unchecked") @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { // sanity check if (this.aggregationFunctions.isEmpty() || this.aggregationFunctions.size() != this.fields.size()) { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java index d3d17c43851a2..20b32b8452a4b 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java @@ -68,7 +68,7 @@ protected Keys getKeys2() { } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { String name = getName() != null ? getName() : function.getClass().getName(); @@ -106,10 +106,7 @@ else if (keys1 instanceof Keys.FieldPositionKeys // set inputs po.setFirstInput(input1); po.setSecondInput(input2); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } + // set dop po.setDegreeOfParallelism(this.getParallelism()); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java index 8a48657b560b0..dbaa7c1fd349c 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java @@ -51,7 +51,7 @@ protected CrossOperator(DataSet input1, DataSet input2, } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator @@ -59,10 +59,7 @@ protected Operator translateToDataFlow(Operator input1, Operator input2) { // set inputs po.setFirstInput(input1); po.setSecondInput(input2); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } + // set dop po.setDegreeOfParallelism(this.getParallelism()); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java index 73ba75a6ae151..c379dde3a96c6 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/DistinctOperator.java @@ -36,8 +36,7 @@ public DistinctOperator(DataSet input, Keys keys) { } @Override - protected Operator translateToDataFlow(Operator input) { - + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { throw new UnsupportedOperationException("NOT IMPLEMENTED"); } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java index 73292d85591da..5028c81c0f570 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java @@ -15,6 +15,7 @@ package eu.stratosphere.api.java.operators; import eu.stratosphere.api.common.operators.Operator; +import eu.stratosphere.api.common.operators.SingleInputOperator; import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.functions.FilterFunction; import eu.stratosphere.api.java.operators.translation.PlanFilterOperator; @@ -39,7 +40,7 @@ public FilterOperator(DataSet input, FilterFunction function) { } @Override - protected Operator translateToDataFlow(Operator input) { + protected SingleInputOperator translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java index 5e77651cbe6b0..4e6b1cc5e52a7 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java @@ -41,17 +41,14 @@ public FlatMapOperator(DataSet input, FlatMapFunction function) { } @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator PlanFlatMapOperator po = new PlanFlatMapOperator(function, name, getInputType(), getResultType()); // set input po.setInput(input); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } + // set dop if(this.getParallelism() > 0) { // use specified dop diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java index b8c95720da1dd..7efbbce1e3082 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java @@ -179,7 +179,7 @@ protected EquiJoin(DataSet input1, DataSet input2, // } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { String name = getName() != null ? getName() : function.getClass().getName(); @@ -217,10 +217,7 @@ else if (super.keys1 instanceof Keys.FieldPositionKeys // set inputs po.setFirstInput(input1); po.setSecondInput(input2); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } + // set dop po.setDegreeOfParallelism(this.getParallelism()); @@ -505,7 +502,7 @@ protected LeftAntiJoin(DataSet input1, DataSet input2, Keys keys1, K } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { throw new UnsupportedOperationException("LeftAntiJoin operator currently not supported."); } } @@ -518,7 +515,7 @@ protected RightAntiJoin(DataSet input1, DataSet input2, Keys keys1, } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { throw new UnsupportedOperationException("RightAntiJoin operator currently not supported."); } } @@ -531,7 +528,7 @@ protected LeftSemiJoin(DataSet input1, DataSet input2, Keys keys1, K } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { // TODO: Runtime support required. Each left tuple may be returned only once. // Special exec strategy (runtime + optimizer) based on hash join required. // Either no duplicates of right side in HT or left tuples removed from HT after first match. @@ -547,7 +544,7 @@ protected RightSemiJoin(DataSet input1, DataSet input2, Keys keys1, } @Override - protected Operator translateToDataFlow(Operator input1, Operator input2) { + protected eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow(Operator input1, Operator input2) { // TODO: Runtime support required. Each right tuple may be returned only once. // Special exec strategy (runtime + optimizer) based on hash join required. // Either no duplicates of left side in HT or right tuples removed from HT after first match. diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java index ace3e73ec5d0f..82c3233ca8b5f 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java @@ -41,21 +41,18 @@ public MapOperator(DataSet input, MapFunction function) { } @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // create operator PlanMapOperator po = new PlanMapOperator(function, name, getInputType(), getResultType()); // set input po.setInput(input); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } + // set dop - if(this.getParallelism() > 0) { + if (getParallelism() > 0) { // use specified dop - po.setDegreeOfParallelism(this.getParallelism()); + po.setDegreeOfParallelism(getParallelism()); } else { // if no dop has been specified, use dop of input operator to enable chaining po.setDegreeOfParallelism(input.getDegreeOfParallelism()); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java index 2555294c77fb3..8add47cb2388c 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/OperatorTranslation.java @@ -16,8 +16,10 @@ import eu.stratosphere.api.common.operators.AbstractUdfOperator; import eu.stratosphere.api.common.operators.BulkIteration; +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; import eu.stratosphere.api.common.operators.GenericDataSink; import eu.stratosphere.api.common.operators.Operator; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.java.BulkIterationResultSet; import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.DeltaIteration; @@ -33,7 +35,6 @@ import java.util.List; import java.util.Map; - /** * */ @@ -84,12 +85,22 @@ else if (dataSet instanceof SingleInputOperator) { // translate the input Operator input = translate(op.getInput()); // translate the operation itself and connect it to the input - dataFlowOp = op.translateToDataFlow(input); + eu.stratosphere.api.common.operators.SingleInputOperator singleInDataFlowOp = op.translateToDataFlow(input); + dataFlowOp = singleInDataFlowOp; - if(dataSet instanceof UdfOperator ) { - Configuration opParams = ((UdfOperator) op).getParameters(); - if(opParams != null) { - dataFlowOp.getParameters().addAll(opParams); + if (dataSet instanceof UdfOperator ) { + SingleInputUdfOperator udfOp = (SingleInputUdfOperator) op; + + // set configuration parameters + Configuration opParams = udfOp.getParameters(); + if (opParams != null) { + singleInDataFlowOp.getParameters().addAll(opParams); + } + + // set the semantic properties + SingleInputSemanticProperties props = udfOp.getSematicProperties(); + if (props != null) { + singleInDataFlowOp.setSemanticProperties(props); } } } @@ -101,14 +112,24 @@ else if (dataSet instanceof TwoInputOperator) { Operator input2 = translate(op.getInput2()); // translate the operation itself and connect it to the inputs - dataFlowOp = op.translateToDataFlow(input1, input2); + eu.stratosphere.api.common.operators.DualInputOperator binaryDataFlowOp = op.translateToDataFlow(input1, input2); + dataFlowOp = binaryDataFlowOp; - // set configuration params - if(dataSet instanceof UdfOperator ) { - Configuration opParams = ((UdfOperator) op).getParameters(); - if(opParams != null) { + + if (dataSet instanceof UdfOperator ) { + TwoInputUdfOperator udfOp = (TwoInputUdfOperator) op; + + // set configuration parameters + Configuration opParams = udfOp.getParameters(); + if (opParams != null) { dataFlowOp.getParameters().addAll(opParams); } + + // set the semantic properties + DualInputSemanticProperties props = udfOp.getSematicProperties(); + if (props != null) { + binaryDataFlowOp.setSemanticProperties(props); + } } } else if (dataSet instanceof BulkIterationResultSet) { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ProjectOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ProjectOperator.java index 41d4a83481296..b651d21414427 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ProjectOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ProjectOperator.java @@ -42,7 +42,7 @@ public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo retur } @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { String name = getName() != null ? getName() : "Projection "+Arrays.toString(fields); // create operator diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java index 359f1e4daad1a..7cf56d79d8eb6 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java @@ -104,7 +104,7 @@ public void setCombinable(boolean combinable) { } @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); @@ -144,10 +144,6 @@ else if (grouper.getKeys() instanceof Keys.FieldPositionKeys) { // set input po.setInput(input); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } // set dop po.setDegreeOfParallelism(this.getParallelism()); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java index e9afe13bbc7ef..82349b1c76a01 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java @@ -67,17 +67,13 @@ public ReduceOperator(Grouping input, ReduceFunction function) { } @Override - protected Operator translateToDataFlow(Operator input) { + protected eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow(Operator input) { String name = getName() != null ? getName() : function.getClass().getName(); // distinguish between grouped reduce and non-grouped reduce if (grouper == null) { // non grouped reduce PlanReduceOperator po = new PlanReduceOperator(function, new int[0], name, getInputType()); - //set semantic properties - if (this.getSematicProperties() != null) { - po.setSemanticProperties(this.getSematicProperties()); - } // set input po.setInput(input); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputOperator.java index 4008e37dfc50e..fbdba360fe966 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputOperator.java @@ -40,7 +40,7 @@ public TypeInformation getInputType() { return this.input.getType(); } - protected abstract eu.stratosphere.api.common.operators.Operator translateToDataFlow( + protected abstract eu.stratosphere.api.common.operators.SingleInputOperator translateToDataFlow( eu.stratosphere.api.common.operators.Operator input); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java index 62e0eae90d21c..3fe7ce2665393 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java @@ -82,9 +82,10 @@ public O withBroadcastSet(DataSet data, String name) { return returnType; } - public O withProperties(String[] constantSet, String constantSetExcept, String readSet) { - SingleInputSemanticProperties props = SemanticPropUtil.getSemanticPropsSingleFromString(constantSet, constantSetExcept, readSet, this.getInputType(), this.getResultType()); + public O withConstantSet(String... constantSet) { + SingleInputSemanticProperties props = SemanticPropUtil.getSemanticPropsSingleFromString(constantSet, null, null, this.getInputType(), this.getResultType()); this.setSemanticProperties(props); + @SuppressWarnings("unchecked") O returnType = (O) this; return returnType; } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputOperator.java index 5779db880468d..42c037127386f 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputOperator.java @@ -60,6 +60,6 @@ public TypeInformation getInput2Type() { * @param input2 The second input of the operation, as a common API operator. * @return The created common API operator. */ - protected abstract eu.stratosphere.api.common.operators.Operator translateToDataFlow( + protected abstract eu.stratosphere.api.common.operators.DualInputOperator translateToDataFlow( eu.stratosphere.api.common.operators.Operator input1, eu.stratosphere.api.common.operators.Operator input2); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java index fb4131b6b28e1..18303916fbde9 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java @@ -85,12 +85,23 @@ public O withBroadcastSet(DataSet data, String name) { return returnType; } - /* - Allows to give specifications about constant sets directly in the code. Null values are allowed for not specified sets. + /** + * Allows to give specifications about constant sets directly in the code. Null values are allowed for not specified sets. */ - public O withProperties(String[] constantSetFirst, String[] constantSetSecond, String constExceptFirst, String constExceptSecond, String readSetFirst, String readSetSecond) { - DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(constantSetFirst, constantSetSecond, - constExceptFirst, constExceptSecond, readSetFirst, readSetSecond, this.getInput1Type(), this.getInput2Type(), this.getResultType()); + @SuppressWarnings("unchecked") + public O withConstantSetFirst(String... constantSetFirst) { + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(constantSetFirst, null, + null, null, null, null, this.getInput1Type(), this.getInput2Type(), this.getResultType()); + this.setSemanticProperties(dsp); + + O returnType = (O) this; + return returnType; + } + + @SuppressWarnings("unchecked") + public O withConstantSetSecond(String... constantSetSecond) { + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(null, constantSetSecond, + null, null, null, null, this.getInput1Type(), this.getInput2Type(), this.getResultType()); this.setSemanticProperties(dsp); O returnType = (O) this; diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java index 5838835bea7da..4c514c83e6ced 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanReduceOperator.java @@ -15,7 +15,9 @@ package eu.stratosphere.api.java.operators.translation; import eu.stratosphere.api.common.functions.GenericReduce; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; import eu.stratosphere.api.common.operators.base.ReduceOperatorBase; +import eu.stratosphere.api.java.functions.FunctionAnnotation; import eu.stratosphere.api.java.functions.ReduceFunction; import eu.stratosphere.api.java.functions.SemanticPropUtil; import eu.stratosphere.api.java.typeutils.TypeInformation; diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/functions/SemanticPropUtilTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/functions/SemanticPropUtilTest.java new file mode 100644 index 0000000000000..439b521b4f9c7 --- /dev/null +++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/functions/SemanticPropUtilTest.java @@ -0,0 +1,198 @@ +/*********************************************************************************************************************** + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.functions; + +import eu.stratosphere.api.common.operators.DualInputSemanticProperties; +import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; +import eu.stratosphere.api.common.operators.util.FieldSet; +import eu.stratosphere.api.java.functions.SemanticPropUtil; +import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.api.java.tuple.Tuple4; +import eu.stratosphere.api.java.typeutils.BasicTypeInfo; +import eu.stratosphere.api.java.typeutils.TupleTypeInfo; +import eu.stratosphere.api.java.typeutils.TypeInformation; +import junit.framework.Assert; + +import org.junit.Test; + +public class SemanticPropUtilTest { + + @Test + public void testSimpleCase() { + String[] constantFields = { "0->0,1", "1->2" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); + + FieldSet fs = sp.getForwardedField(0); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(0)); + Assert.assertTrue(fs.contains(1)); + + fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(2)); + } + + @Test + public void testSimpleCaseWildCard() { + String[] constantFields = { "*" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(1)); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(2)); + + fs = sp.getForwardedField(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(0)); + } + + @Test + public void testSimpleCaseWildCard2() { + String[] constantFields = { "1->*" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 3); + Assert.assertTrue(fs.contains(0)); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + Assert.assertTrue(sp.getForwardedField(0) == null); + Assert.assertTrue(sp.getForwardedField(2) == null); + } + + @Test + public void testConstantFieldsExcept() { + String[] constantFieldsExcept = { "1" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(null, constantFieldsExcept, null, type, type); + + FieldSet fs = sp.getForwardedField(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(0)); + + fs = sp.getForwardedField(1); + Assert.assertTrue(fs == null); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(2)); + } + + @Test + public void testReadFields() { + String[] readFields = { "1, 2" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(null, null, readFields, type, type); + + FieldSet fs = sp.getReadFields(); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(2)); + Assert.assertTrue(fs.contains(1)); + } + + @Test + public void testSimpleCaseDual() { + String[] constantFieldsFirst = { "1->1,2", "2->3" }; + String[] constantFieldsSecond = { "1->1,2", "2->3" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(constantFieldsFirst, constantFieldsSecond, null, + null, null, null, type, type, type); + + FieldSet fs = dsp.getForwardedField1(1); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + + fs = dsp.getForwardedField1(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(3)); + } + + @Test + public void testFieldsExceptDual() { + String[] constantFieldsFirstExcept = { "1,2" }; + String[] constantFieldsSecond = { "0->1" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(null, constantFieldsSecond, + constantFieldsFirstExcept, null, null, null, type, type, type); + + FieldSet fs = dsp.getForwardedField1(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(0)); + + fs = dsp.getForwardedField1(1); + Assert.assertTrue(fs == null); + + fs = dsp.getForwardedField1(2); + Assert.assertTrue(fs == null); + + fs = dsp.getForwardedField2(0); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(1)); + } + + @Test + public void testStringParse1() { + String[] constantFields = { " 1-> 1 , 2", "2 ->3" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); + + FieldSet fs = sp.getForwardedField(1); + Assert.assertTrue(fs.size() == 2); + Assert.assertTrue(fs.contains(1)); + Assert.assertTrue(fs.contains(2)); + + fs = sp.getForwardedField(2); + Assert.assertTrue(fs.size() == 1); + Assert.assertTrue(fs.contains(3)); + } + + @Test + public void testStringParse2() { + String[] constantFields = { "notValid" }; + + TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + + try { + SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); + Assert.fail(); + } catch (Exception e) { + // good + } + } +} diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 31f94956d1540..4b832638063c7 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -404,6 +404,12 @@ public static void main(String[] args) throws IOException { System.err.println("Configuration "+GlobalConfiguration.getConfiguration()); LOG.info("Current user "+UserGroupInformation.getCurrentUser().getShortUserName()); + { + // log the available JVM memory + long maxMemoryMiBytes = Runtime.getRuntime().maxMemory() >>> 20; + LOG.info("Starting TaskManager in a JVM with " + maxMemoryMiBytes + " MiBytes maximum heap size."); + } + // Create a new task manager object try { new TaskManager(); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java deleted file mode 100644 index 2f83063640414..0000000000000 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/semanticprops/SemanticPropUtilTest.java +++ /dev/null @@ -1,187 +0,0 @@ -/*********************************************************************************************************************** - * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed 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 eu.stratosphere.test.semanticprops; - -import eu.stratosphere.api.common.operators.DualInputSemanticProperties; -import eu.stratosphere.api.common.operators.SingleInputSemanticProperties; -import eu.stratosphere.api.common.operators.util.FieldSet; -import eu.stratosphere.api.java.functions.SemanticPropUtil; -import eu.stratosphere.api.java.tuple.Tuple3; -import eu.stratosphere.api.java.typeutils.BasicTypeInfo; -import eu.stratosphere.api.java.typeutils.TupleTypeInfo; -import eu.stratosphere.api.java.typeutils.TypeInformation; -import junit.framework.Assert; -import org.junit.Test; - -public class SemanticPropUtilTest { - - @Test - public void testSimpleCase() { - String[] constantFields = {"1->1,2", "2->3"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); - - FieldSet fs = sp.getForwardedField(1); - Assert.assertTrue(fs.size() == 2); - Assert.assertTrue(fs.contains(1)); - Assert.assertTrue(fs.contains(2)); - - fs = sp.getForwardedField(2); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(3)); - } - - @Test - public void testSimpleCaseWildCard() { - String[] constantFields = {"*"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); - - FieldSet fs = sp.getForwardedField(1); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(1)); - - fs = sp.getForwardedField(2); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(2)); - - fs = sp.getForwardedField(0); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(0)); - } - - @Test - public void testSimpleCaseWildCard2() { - String[] constantFields = {"1->*"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); - - FieldSet fs = sp.getForwardedField(1); - Assert.assertTrue(fs.size() == 3); - Assert.assertTrue(fs.contains(0)); - Assert.assertTrue(fs.contains(1)); - Assert.assertTrue(fs.contains(2)); - Assert.assertTrue(sp.getForwardedField(0) == null); - Assert.assertTrue(sp.getForwardedField(2) == null); - } - - @Test - public void testConstantFieldsExcept() { - String constantFieldsExcept = "1"; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(null, constantFieldsExcept, null,type, type); - - FieldSet fs = sp.getForwardedField(0); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(0)); - - fs = sp.getForwardedField(1); - Assert.assertTrue(fs == null); - - fs = sp.getForwardedField(2); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(2)); - } - - @Test - public void testReadFields() { - String readFields = "1, 2"; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(null, null, readFields,type, type); - - FieldSet fs = sp.getReadFields(); - Assert.assertTrue(fs.size() == 2); - Assert.assertTrue(fs.contains(2)); - Assert.assertTrue(fs.contains(1)); - } - - @Test - public void testSimpleCaseDual() { - String[] constantFieldsFirst = {"1->1,2", "2->3"}; - String[] constantFieldsSecond = {"1->1,2", "2->3"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(constantFieldsFirst, constantFieldsSecond, null, null, null, null, type, type, type); - - FieldSet fs = dsp.getForwardedField1(1); - Assert.assertTrue(fs.size() == 2); - Assert.assertTrue(fs.contains(1)); - Assert.assertTrue(fs.contains(2)); - - fs = dsp.getForwardedField1(2); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(3)); - } - - @Test - public void testFieldsExceptDual() { - String constantFieldsFirstExcept = "1,2"; - String[] constantFieldsSecond = {"0->1"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(null, constantFieldsSecond, constantFieldsFirstExcept, null, null, null, type, type, type); - - FieldSet fs = dsp.getForwardedField1(0); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(0)); - - fs = dsp.getForwardedField1(1); - Assert.assertTrue(fs == null); - - fs = dsp.getForwardedField1(2); - Assert.assertTrue(fs == null); - - fs = dsp.getForwardedField2(0); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(1)); - } - - @Test - public void testStringParse1() { - String[] constantFields = {" 1-> 1 , 2", "2 ->3"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null,type, type); - - FieldSet fs = sp.getForwardedField(1); - Assert.assertTrue(fs.size() == 2); - Assert.assertTrue(fs.contains(1)); - Assert.assertTrue(fs.contains(2)); - - fs = sp.getForwardedField(2); - Assert.assertTrue(fs.size() == 1); - Assert.assertTrue(fs.contains(3)); - } - - @Test - public void testStringParse2() { - String[] constantFields = {"notValid"}; - - TypeInformation type = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - - try { - SingleInputSemanticProperties sp = SemanticPropUtil.getSemanticPropsSingleFromString(constantFields, null, null, type, type); - } catch (Exception e) { - return; - } - Assert.fail(); - } - - -} From ce53a8322fa1943d1d83955c5d7504b4c12ac78b Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 18:03:21 +0200 Subject: [PATCH 040/182] Removed combine methods from CoGroup function and some minor improvements. --- .../common/functions/GenericCoGrouper.java | 23 -------- .../api/java/functions/CoGroupFunction.java | 55 +------------------ .../PlanUnwrappingCoGroupOperator.java | 12 +--- .../record/functions/CoGroupFunction.java | 35 ------------ .../record/operators/CoGroupOperator.java | 18 +----- .../CoGroupConnectedComponentsITCase.java | 16 ------ .../CustomCompensatableDotProductCoGroup.java | 7 +-- .../graph/ConnectedComponentsWithCoGroup.java | 17 +----- 8 files changed, 9 insertions(+), 174 deletions(-) diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/functions/GenericCoGrouper.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/functions/GenericCoGrouper.java index 5caff99415d6d..4d7f59039abab 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/functions/GenericCoGrouper.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/functions/GenericCoGrouper.java @@ -31,27 +31,4 @@ public interface GenericCoGrouper extends Function { */ void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception; - /** - * This method must be overridden by CoGoup UDFs that want to make use of the combining feature - * on their first input. In addition, the extending class must be annotated as CombinableFirst. - * - * The use of the combiner is typically a pre-reduction of the data. - * - * @param records The records to be combined. - * @param out The collector to write the result to. - * - */ - V1 combineFirst(Iterator records) throws Exception; - - /** - * This method must be overridden by CoGoup UDFs that want to make use of the combining feature - * on their second input. In addition, the extending class must be annotated as CombinableSecond. - * - * The use of the combiner is typically a pre-reduction of the data. - * - * @param records The records to be combined. - * @param out The collector to write the result to. - * - */ - V2 combineSecond(Iterator records) throws Exception; } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/CoGroupFunction.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/CoGroupFunction.java index 5ea597763ec5e..d9ea8e330d363 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/CoGroupFunction.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/CoGroupFunction.java @@ -14,14 +14,11 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java.functions; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; import java.util.Iterator; import eu.stratosphere.api.common.functions.AbstractFunction; import eu.stratosphere.api.common.functions.GenericCoGrouper; +import eu.stratosphere.api.java.operators.Keys; import eu.stratosphere.util.Collector; /** @@ -35,7 +32,7 @@ * * set1.coGroup(set2).where().equalTo().with(new MyCoGroupFunction()); * - * The keys can be defined in various ways, such as through field names, tuple field positions, or key extractors. + * The keys can be defined through tuple field positions or key extractors. * See {@link Keys} for details. *

* Some keys may only be contained in one of the two original data sets. In that case, the CoGroup function is invoked @@ -48,57 +45,11 @@ public abstract class CoGroupFunction extends AbstractFunction im /** * The core method of the CoGroupFunction. This method is called for each pair of groups that have the same - * key. The elements of the groups are returned by the repestive iterators. + * key. The elements of the groups are returned by the respective iterators. * * It is possible that one of the two groups is empty, in which case the respective iterator has no elements. */ @Override public abstract void coGroup(Iterator first, Iterator second, Collector out) throws Exception; - - - /** - * This function is not implemented by default. It must be overridden, if the function declares through the - * {@link CombinableFirst} annotation that the first input is combinable. - * - * @see eu.stratosphere.api.common.functions.GenericCoGrouper#combineFirst(java.util.Iterator) - */ - @Override - public IN1 combineFirst(Iterator records) throws Exception { - throw new UnsupportedOperationException("combineFirst() is not overridden by this UDF. " + - "Using the CombinableFirst annotation requires that this method is overridden."); - } - /** - * This function is not implemented by default. It must be overridden, if the function declares through the - * {@link CombinableFirst} annotation that the first input is combinable. - * - * @see eu.stratosphere.api.common.functions.GenericCoGrouper#combineFirst(java.util.Iterator) - */ - @Override - public IN2 combineSecond(Iterator records) throws Exception { - throw new UnsupportedOperationException("combineSecond() is not overridden by this UDF. " + - "Using the CombinableSecond annotation requires that this method is overridden."); - } - - // -------------------------------------------------------------------------------------------- - // Annotations for to declare individual CoGroup inputs combinable - // -------------------------------------------------------------------------------------------- - - /** - * This marker interface can be added to a CoGroup function implementation. It declares its first input - * combinable. Similar as for the {@link GroupReduceFunction} function, the framework may invoke the - * {@link CoGroupFunction#combineFirst(Iterator)} function to reduce the data volume early. - */ - @Retention(RetentionPolicy.RUNTIME) - @Target(ElementType.TYPE) - public static @interface CombinableFirst {}; - - /** - * This marker interface can be added to a CoGroup function implementation. It declares its second input - * combinable. Similar as for the {@link GroupReduceFunction} function, the framework may invoke the - * {@link CoGroupFunction#combineSecond(Iterator)} function to reduce the data volume early. - */ - @Retention(RetentionPolicy.RUNTIME) - @Target(ElementType.TYPE) - public static @interface CombinableSecond {}; } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java index e1d4e17f3e609..ecef514760568 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java @@ -103,17 +103,7 @@ private TupleUnwrappingCoGrouper(CoGroupFunction wrapped) { public void coGroup(Iterator> records1, Iterator> records2, Collector out) throws Exception { this.wrappedFunction.coGroup(new UnwrappingKeyIterator(records1), new UnwrappingKeyIterator(records2), out); } - - - @Override - public Tuple2 combineFirst(Iterator> records) throws Exception { - throw new UnsupportedOperationException(); - } - - @Override - public Tuple2 combineSecond(Iterator> records) throws Exception { - throw new UnsupportedOperationException(); - } + } public static class UnwrappingKeyIterator implements Iterator { diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/functions/CoGroupFunction.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/functions/CoGroupFunction.java index 41fb8935e4180..3a81d34402ce8 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/functions/CoGroupFunction.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/functions/CoGroupFunction.java @@ -43,39 +43,4 @@ public abstract class CoGroupFunction extends AbstractFunction implements Generi @Override public abstract void coGroup(Iterator records1, Iterator records2, Collector out) throws Exception; - /** - * This method must be overridden by CoGoup UDFs that want to make use of the combining feature - * on their first input. In addition, the extending class must be annotated as CombinableFirst. - *

- * The use of the combiner is typically a pre-reduction of the data. - * - * @param records The records to be combined. - * @param out The collector to write the result to. - * - * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the - * runtime catches an exception, it aborts the combine task and lets the fail-over logic - * decide whether to retry the combiner execution. - */ - @Override - public Record combineFirst(Iterator records) throws Exception { - throw new UnsupportedOperationException(); - } - - /** - * This method must be overridden by CoGoup UDFs that want to make use of the combining feature - * on their second input. In addition, the extending class must be annotated as CombinableSecond. - *

- * The use of the combiner is typically a pre-reduction of the data. - * - * @param records The records to be combined. - * @param out The collector to write the result to. - * - * @throws Exception Implementations may forward exceptions, which are caught by the runtime. When the - * runtime catches an exception, it aborts the combine task and lets the fail-over logic - * decide whether to retry the combiner execution. - */ - @Override - public Record combineSecond(Iterator records) throws Exception { - throw new UnsupportedOperationException(); - } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CoGroupOperator.java index c50264278d9a3..159cf3404ddff 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CoGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/operators/CoGroupOperator.java @@ -13,10 +13,6 @@ package eu.stratosphere.api.java.record.operators; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -106,22 +102,14 @@ public Class>[] getKeyClasses() { @Override public boolean isCombinableFirst() { - return super.isCombinableFirst() || getUserCodeWrapper().getUserCodeAnnotation(CombinableFirst.class) != null; + return false; } @Override public boolean isCombinableSecond() { - return super.isCombinableSecond() || getUserCodeWrapper().getUserCodeAnnotation(CombinableSecond.class) != null; + return false; } - @Retention(RetentionPolicy.RUNTIME) - @Target(ElementType.TYPE) - public static @interface CombinableFirst {}; - - @Retention(RetentionPolicy.RUNTIME) - @Target(ElementType.TYPE) - public static @interface CombinableSecond {}; - // -------------------------------------------------------------------------------------------- @@ -168,7 +156,7 @@ protected Builder(UserCodeWrapper udf, Class> } /** - * Creates a Builder with the provided {@link JoinFunction} implementation. This method is intended + * Creates a Builder with the provided {@link CoGroupFunction} implementation. This method is intended * for special case sub-types only. * * @param udf The {@link CoGroupFunction} implementation for this CoGroup operator. diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java index 50c9fad48d87b..f299f2e8cfc40 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/CoGroupConnectedComponentsITCase.java @@ -27,7 +27,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat; import eu.stratosphere.api.java.record.io.CsvOutputFormat; import eu.stratosphere.api.java.record.operators.CoGroupOperator; -import eu.stratosphere.api.java.record.operators.CoGroupOperator.CombinableFirst; import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; @@ -77,7 +76,6 @@ protected void postSubmit() throws Exception { // -------------------------------------------------------------------------------------------- - @CombinableFirst @ConstantFieldsFirst(0) @ConstantFieldsSecond(0) public static final class MinIdAndUpdate extends CoGroupFunction implements Serializable { @@ -108,20 +106,6 @@ public void coGroup(Iterator candidates, Iterator current, Colle out.collect(old); } } - - @Override - public Record combineFirst(Iterator records) { - Record next = null; - long min = Long.MAX_VALUE; - while (records.hasNext()) { - next = records.next(); - min = Math.min(min, next.getField(1, LongValue.class).getValue()); - } - - newComponentId.setValue(min); - next.setField(1, newComponentId); - return next; - } } @SuppressWarnings("unchecked") diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java index a24a75d958955..927d380f646bb 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java @@ -119,10 +119,5 @@ public void close() throws Exception { aggregator.reset(); } } - - @Override - public VertexWithRankAndDangling combineFirst(Iterator records) { throw new UnsupportedOperationException(); } - - @Override - public VertexWithRank combineSecond(Iterator records) { throw new UnsupportedOperationException(); } + } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java index 923e4ad50f7f8..c50301827f335 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/ConnectedComponentsWithCoGroup.java @@ -18,9 +18,9 @@ import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.Program; +import eu.stratosphere.api.common.operators.DeltaIteration; import eu.stratosphere.api.common.operators.FileDataSink; import eu.stratosphere.api.common.operators.FileDataSource; -import eu.stratosphere.api.common.operators.DeltaIteration; import eu.stratosphere.api.java.record.functions.CoGroupFunction; import eu.stratosphere.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst; import eu.stratosphere.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond; @@ -29,7 +29,6 @@ import eu.stratosphere.api.java.record.operators.CoGroupOperator; import eu.stratosphere.api.java.record.operators.JoinOperator; import eu.stratosphere.api.java.record.operators.MapOperator; -import eu.stratosphere.api.java.record.operators.CoGroupOperator.CombinableFirst; import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap; import eu.stratosphere.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin; import eu.stratosphere.types.LongValue; @@ -44,7 +43,6 @@ public class ConnectedComponentsWithCoGroup implements Program { private static final long serialVersionUID = 1L; - @CombinableFirst @ConstantFieldsFirst(0) @ConstantFieldsSecond(0) public static final class MinIdAndUpdate extends CoGroupFunction implements Serializable { @@ -76,19 +74,6 @@ public void coGroup(Iterator candidates, Iterator current, Colle } } - @Override - public Record combineFirst(Iterator records) { - Record next = null; - long min = Long.MAX_VALUE; - while (records.hasNext()) { - next = records.next(); - min = Math.min(min, next.getField(1, LongValue.class).getValue()); - } - - newComponentId.setValue(min); - next.setField(1, newComponentId); - return next; - } } @SuppressWarnings("unchecked") From a589fa3348648f723019a2e9c3cdcebdc826f2c9 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 May 2014 23:25:11 +0200 Subject: [PATCH 041/182] Improved Java examples --- .../example/java/clustering/KMeans.java | 5 ++++- .../example/java/graph/ConnectedComponents.java | 14 +++++++++----- .../example/java/graph/EnumTrianglesBasic.java | 1 + .../example/java/graph/EnumTrianglesOpt.java | 1 + .../example/java/graph/PageRankBasic.java | 6 ++++-- .../example/java/relational/WebLogAnalysis.java | 2 ++ .../example/java/wordcount/WordCount.java | 2 +- 7 files changed, 22 insertions(+), 9 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java index 4ebeed87b1503..bbf5bf2f2d737 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java @@ -38,7 +38,8 @@ * Subsequently, each cluster center is moved to the center (mean) of all points that have been assigned to it. * The moved cluster centers are fed into the next iteration. * The algorithm terminates after a fixed number of iterations (as in this implementation) - * or if cluster centers do not (significantly) move in an iteration. + * or if cluster centers do not (significantly) move in an iteration.
+ * This is the Wikipedia entry for the K-Means Clustering algorithm. * *

* This implementation works on two-dimensional data points.
@@ -290,6 +291,8 @@ private static void parseParameters(String[] programArguments) { } else { System.out.println("Executing K-Means example with default parameters and built-in default data."); System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); + System.out.println(" We provide a data generator to create synthetic input files for this program."); System.out.println(" Usage: KMeans "); } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java index 6f6b9894baf7c..e3b2c77c61798 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java @@ -31,15 +31,18 @@ /** * An implementation of the connected components algorithm, using a delta iteration. - * Initially, the algorithm assigns each vertex its own ID. After the algorithm has completed, all vertices in the - * same component will have the same id. In each step, a vertex picks the minimum of its own ID and its - * neighbors' IDs, as its new ID. + * + *

+ * Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the minimum of its own ID and its + * neighbors' IDs, as its new ID and tells its neighbors about its new ID. After the algorithm has completed, all vertices in the + * same component will have the same ID. + * *

- * A vertex whose component did not change needs not propagate its information in the next step. Because of that, + * A vertex whose component ID did not change needs not propagate its information in the next step. Because of that, * the algorithm is easily expressible via a delta iteration. We here model the solution set as the vertices with * their current component ids, and the workset as the changed vertices. Because we see all vertices initially as * changed, the initial workset and the initial solution set are identical. Also, the delta to the solution set - * is consequently also the next workset. + * is consequently also the next workset.
* *

* Input files are plain text files and must be formatted as follows: @@ -180,6 +183,7 @@ private static void parseParameters(String[] programArguments) { } else { System.out.println("Executing Connected Components example with default parameters and built-in default data."); System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); System.out.println(" Usage: ConnectedComponents "); } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java index 4337fd48e5ae1..4c491fc1faae5 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesBasic.java @@ -198,6 +198,7 @@ private static void parseParameters(String[] args) { } else { System.out.println("Executing Enum Triangles Basic example with built-in default data."); System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); System.out.println(" Usage: EnumTriangleBasic "); } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java index 57ff558bf2b8b..33683e5e8e9e7 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/EnumTrianglesOpt.java @@ -322,6 +322,7 @@ private static void parseParameters(String[] args) { } else { System.out.println("Executing Enum Triangles Opt example with built-in default data."); System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); System.out.println(" Usage: EnumTriangleBasic "); } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java index cf66cce012bb9..e25206c5e6b1d 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java @@ -31,7 +31,7 @@ import eu.stratosphere.util.Collector; /** - * A basic implementation of the page rank algorithm using a bulk iteration. + * A basic implementation of the Page Rank algorithm using a bulk iteration. * *

* This implementation requires a set of pages (vertices) with associated ranks and a set @@ -39,7 +39,8 @@ * In each iteration, the rank of every page is evenly distributed to all pages it points to. * Each page collects the partial ranks of all pages that point to it, sums them up, and applies a dampening factor to the sum. * The result is the new rank of the page. A new iteration is started with the new ranks of all pages. - * This implementation terminates after a fixed number of iterations. + * This implementation terminates after a fixed number of iterations.
+ * This is the Wikipedia entry for the Page Rank algorithm. * *

* Input files are plain text files and must be formatted as follows: @@ -216,6 +217,7 @@ private static void parseParameters(String[] args) { } else { System.out.println("Executing PageRank Basic example with default parameters and built-in default data."); System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); System.out.println(" Usage: PageRankBasic "); numVertices = PageRankData.getNumberOfPages(); diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java index d7eb4a95f3e36..562f0f388c391 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/WebLogAnalysis.java @@ -275,6 +275,8 @@ private static void parseParameters(String[] args) { } else { System.out.println("Executing WebLog Analysis example with built-in default data."); System.out.println(" Provide parameters to read input data from files."); + System.out.println(" See the documentation for the correct format of input files."); + System.out.println(" We provide a data generator to create synthetic input files for this program."); System.out.println(" Usage: WebLogAnalysis "); } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java index 40c2f21d73d35..6db69863321df 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java @@ -27,7 +27,7 @@ * over text files. * *

- * The input is plain text files. + * The input is a plain text file with lines separated by newline characters. * *

* This example shows how to: From bbc055b1710338b5e400b9779e4ebda90cdcf0c8 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Fri, 16 May 2014 15:17:18 +0200 Subject: [PATCH 042/182] Completed, updated, and improved JavaDocs of Java API --- .../eu/stratosphere/api/java/DataSet.java | 84 +++++++++++------ .../api/java/operators/CoGroupOperator.java | 91 ++++++++++++++++--- .../api/java/operators/CrossOperator.java | 27 +++++- .../api/java/operators/JoinOperator.java | 38 ++++---- .../operators/SingleInputUdfOperator.java | 18 ++++ .../java/operators/TwoInputUdfOperator.java | 35 ++++++- 6 files changed, 235 insertions(+), 58 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java index ec4bfd3337cd6..aadef1b19c96e 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java @@ -31,6 +31,7 @@ import eu.stratosphere.api.java.operators.AggregateOperator; import eu.stratosphere.api.java.operators.CoGroupOperator; import eu.stratosphere.api.java.operators.CoGroupOperator.CoGroupOperatorSets; +import eu.stratosphere.api.java.operators.CrossOperator.DefaultCross; import eu.stratosphere.api.java.operators.CrossOperator; import eu.stratosphere.api.java.operators.CustomUnaryOperation; import eu.stratosphere.api.java.operators.DataSink; @@ -46,6 +47,7 @@ import eu.stratosphere.api.java.operators.ReduceOperator; import eu.stratosphere.api.java.operators.UnionOperator; import eu.stratosphere.api.java.tuple.Tuple; +import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.typeutils.InputTypeConfigurable; import eu.stratosphere.api.java.typeutils.TypeInformation; import eu.stratosphere.core.fs.FileSystem.WriteMode; @@ -418,23 +420,41 @@ public CoGroupOperator.CoGroupOperatorSets coGroup(DataSet other) { // Cross // -------------------------------------------------------------------------------------------- + /** + * Continues a Join transformation and defines the {@link Tuple} fields of the second join + * {@link DataSet} that should be used as join keys.
+ * Note: Fields can only be selected as join keys on Tuple DataSets.
+ * + * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with + * the element of the first input being the first field of the tuple and the element of the + * second input being the second field of the tuple. + * + * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys. + * @return A DefaultJoin that represents the joined DataSet. + */ + /** * Initiates a Cross transformation.
* A Cross transformation combines the elements of two * {@link DataSet DataSets} into one DataSet. It builds all pair combinations of elements of - * both DataSets, i.e., it builds a Cartesian product, and calls a {@link CrossFunction} for - * each pair of elements.
- * The CrossFunction returns a exactly one element for each pair of input elements.
- * This method returns a {@link CrossOperatorSets} on which - * {@link CrossOperatorSets#with()} needs to be called to define the CrossFunction that - * is applied. + * both DataSets, i.e., it builds a Cartesian product. + * + *

+ * The resulting {@link DefaultCross} wraps each pair of crossed elements into a {@link Tuple2}, with + * the element of the first input being the first field of the tuple and the element of the + * second input being the second field of the tuple. + * + *

+ * Call {@link DefaultCross.with(CrossFunction)} to define a {@link CrossFunction} which is called for + * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.
* * @param other The other DataSet with which this DataSet is crossed. - * @return A CrossOperatorSets to continue the definition of the Cross transformation. + * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements. * + * @see DefaultCross * @see CrossFunction - * @see CrossOperatorSets * @see DataSet + * @see Tuple2 */ public CrossOperator.DefaultCross cross(DataSet other) { return new CrossOperator.DefaultCross(this, other); @@ -444,21 +464,26 @@ public CrossOperator.DefaultCross cross(DataSet other) { * Initiates a Cross transformation.
* A Cross transformation combines the elements of two * {@link DataSet DataSets} into one DataSet. It builds all pair combinations of elements of - * both DataSets, i.e., it builds a Cartesian product, and calls a {@link CrossFunction} for - * each pair of elements.
- * The CrossFunction returns a exactly one element for each pair of input elements.
+ * both DataSets, i.e., it builds a Cartesian product. * This method also gives the hint to the optimizer that the second DataSet to cross is much - * smaller than the first one.
- * This method returns a {@link CrossOperatorSets CrossOperatorSet} on which - * {@link CrossOperatorSets#with()} needs to be called to define the CrossFunction that - * is applied. + * smaller than the first one. + * + *

+ * The resulting {@link DefaultCross} wraps each pair of crossed elements into a {@link Tuple2}, with + * the element of the first input being the first field of the tuple and the element of the + * second input being the second field of the tuple. + * + *

+ * Call {@link DefaultCross.with(CrossFunction)} to define a {@link CrossFunction} which is called for + * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.
* * @param other The other DataSet with which this DataSet is crossed. - * @return A CrossOperatorSets to continue the definition of the Cross transformation. + * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements. * + * @see DefaultCross * @see CrossFunction - * @see CrossOperatorSets * @see DataSet + * @see Tuple2 */ public CrossOperator.DefaultCross crossWithTiny(DataSet other) { return new CrossOperator.DefaultCross(this, other); @@ -468,21 +493,26 @@ public CrossOperator.DefaultCross crossWithTiny(DataSet other) { * Initiates a Cross transformation.
* A Cross transformation combines the elements of two * {@link DataSet DataSets} into one DataSet. It builds all pair combinations of elements of - * both DataSets, i.e., it builds a Cartesian product, and calls a {@link CrossFunction} for - * each pair of elements.
- * The CrossFunction returns a exactly one element for each pair of input elements.
+ * both DataSets, i.e., it builds a Cartesian product. * This method also gives the hint to the optimizer that the second DataSet to cross is much - * larger than the first one.
- * This method returns a {@link CrossOperatorSets CrossOperatorSet} on which - * {@link CrossOperatorSets#with()} needs to be called to define the CrossFunction that - * is applied. + * larger than the first one. + * + *

+ * The resulting {@link DefaultCross} wraps each pair of crossed elements into a {@link Tuple2}, with + * the element of the first input being the first field of the tuple and the element of the + * second input being the second field of the tuple. + * + *

+ * Call {@link DefaultCross.with(CrossFunction)} to define a {@link CrossFunction} which is called for + * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.
* * @param other The other DataSet with which this DataSet is crossed. - * @return A CrossOperatorSets to continue the definition of the Cross transformation. + * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements. * + * @see DefaultCross * @see CrossFunction - * @see CrossOperatorSets * @see DataSet + * @see Tuple2 */ public CrossOperator.DefaultCross crossWithHuge(DataSet other) { return new CrossOperator.DefaultCross(this, other); @@ -766,7 +796,7 @@ public DataSink write(FileOutputFormat outputFormat, String filePath, Writ } /** - * Processes a DataSet using an {@link OutputFormat}. This method adds a data sink to the program. + * Emits a DataSet using an {@link OutputFormat}. This method adds a data sink to the program. * Programs may have multiple data sinks. A DataSet may also have multiple consumers (data sinks * or transformations) at the same time. * diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java index 20b32b8452a4b..1146e579387d8 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java @@ -26,13 +26,20 @@ import eu.stratosphere.api.java.operators.translation.PlanMapOperator; import eu.stratosphere.api.java.operators.translation.PlanUnwrappingCoGroupOperator; import eu.stratosphere.api.java.operators.translation.TupleKeyExtractingMapper; +import eu.stratosphere.api.java.tuple.Tuple; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.typeutils.TupleTypeInfo; import eu.stratosphere.api.java.typeutils.TypeExtractor; import eu.stratosphere.api.java.typeutils.TypeInformation; /** - * + * A {@link DataSet} that is the result of a CoGroup transformation. + * + * @param The type of the first input DataSet of the CoGroup transformation. + * @param The type of the second input DataSet of the CoGroup transformation. + * @param The type of the result of the CoGroup transformation. + * + * @see DataSet */ public class CoGroupOperator extends TwoInputUdfOperator> { @@ -266,6 +273,14 @@ private static PlanUnwrappingCoGroupOperator tr // Builder classes for incremental construction // -------------------------------------------------------------------------------------------- + /** + * Intermediate step of a CoGroup transformation.
+ * To continue the CoGroup transformation, select the grouping key of the first input {@link DataSet} by calling + * {@link CoGroupOperatorSets#where(int...)} or {@link CoGroupOperatorSets#where(KeySelector)}. + * + * @param The type of the first input DataSet of the CoGroup transformation. + * @param The type of the second input DataSet of the CoGroup transformation. + */ public static final class CoGroupOperatorSets { private final DataSet input1; @@ -280,20 +295,46 @@ public CoGroupOperatorSets(DataSet input1, DataSet input2) { this.input2 = input2; } + /** + * Continues a CoGroup transformation.
+ * Defines the {@link Tuple} fields of the first co-grouped {@link DataSet} that should be used as grouping keys.
+ * Note: Fields can only be selected as grouping keys on Tuple DataSets.
+ * + * @param fields The indexes of the Tuple fields of the first co-grouped DataSets that should be used as keys. + * @return An incomplete CoGroup transformation. + * Call {@link CoGroupOperatorSetsPredicate#equalTo()} to continue the CoGroup. + * + * @see Tuple + * @see DataSet + */ public CoGroupOperatorSetsPredicate where(int... fields) { return new CoGroupOperatorSetsPredicate(new Keys.FieldPositionKeys(fields, input1.getType())); } + /** + * Continues a CoGroup transformation and defines a {@link KeySelector} function for the first co-grouped {@link DataSet}.
+ * The KeySelector function is called for each element of the first DataSet and extracts a single + * key value on which the DataSet is grouped.
+ * + * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is grouped. + * @return An incomplete CoGroup transformation. + * Call {@link CoGroupOperatorSetsPredicate#equalTo()} to continue the CoGroup. + * + * @see KeySelector + * @see DataSet + */ public CoGroupOperatorSetsPredicate where(KeySelector keyExtractor) { return new CoGroupOperatorSetsPredicate(new Keys.SelectorFunctionKeys(keyExtractor, input1.getType())); } - public CoGroupOperatorSetsPredicate where(String keyExpression) { - return new CoGroupOperatorSetsPredicate(new Keys.ExpressionKeys(keyExpression, input1.getType())); - } - // ---------------------------------------------------------------------------------------- + /** + * Intermediate step of a CoGroup transformation.
+ * To continue the CoGroup transformation, select the grouping key of the second input {@link DataSet} by calling + * {@link CoGroupOperatorSetsPredicate#equalTo(int...)} or {@link CoGroupOperatorSetsPredicate#equalTo(KeySelector)}. + * + */ public final class CoGroupOperatorSetsPredicate { private final Keys keys1; @@ -310,21 +351,39 @@ private CoGroupOperatorSetsPredicate(Keys keys1) { this.keys1 = keys1; } - + /** + * Continues a CoGroup transformation and defines the {@link Tuple} fields of the second co-grouped + * {@link DataSet} that should be used as grouping keys.
+ * Note: Fields can only be selected as grouping keys on Tuple DataSets.
+ * + * @param fields The indexes of the Tuple fields of the second co-grouped DataSet that should be used as keys. + * @return An incomplete CoGroup transformation. + * Call {@link CoGroupOperatorWithoutFunction#with(CoGroupFunction))} to finalize the CoGroup transformation. + */ public CoGroupOperatorWithoutFunction equalTo(int... fields) { return createCoGroupOperator(new Keys.FieldPositionKeys(fields, input2.getType())); } + /** + * Continues a CoGroup transformation and defines a {@link KeySelector} function for the second co-grouped {@link DataSet}.
+ * The KeySelector function is called for each element of the second DataSet and extracts a single + * key value on which the DataSet is grouped.
+ * + * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is grouped. + * @return An incomplete CoGroup transformation. + * Call {@link CoGroupOperatorWithoutFunction#with(CoGroupFunction))} to finalize the CoGroup transformation. + */ public CoGroupOperatorWithoutFunction equalTo(KeySelector keyExtractor) { return createCoGroupOperator(new Keys.SelectorFunctionKeys(keyExtractor, input2.getType())); } - public CoGroupOperatorWithoutFunction equalTo(String keyExpression) { - return createCoGroupOperator(new Keys.ExpressionKeys(keyExpression, input2.getType())); - } - - + /** + * Intermediate step of a CoGroup transformation.
+ * To continue the CoGroup transformation, provide a {@link CoGroupFunction} by calling + * {@link CoGroupOperatorWithoutFunction#with(CoGroupFunction))}. + * + */ private CoGroupOperatorWithoutFunction createCoGroupOperator(Keys keys2) { if (keys2 == null) { throw new NullPointerException(); @@ -356,6 +415,16 @@ private CoGroupOperatorWithoutFunction(Keys keys2) { this.keys2 = keys2; } + /** + * Finalizes a CoGroup transformation by applying a {@link CoGroupFunction} to groups of elements with identical keys.
+ * Each CoGroupFunction call returns an arbitrary number of keys. + * + * @param function The CoGroupFunction that is called for all groups of elements with identical keys. + * @return An CoGroupOperator that represents the co-grouped result DataSet. + * + * @see CoGroupFunction + * @see DataSet + */ public CoGroupOperator with(CoGroupFunction function) { TypeInformation returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType()); return new CoGroupOperator(input1, input2, keys1, keys2, function, returnType); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java index dbaa7c1fd349c..c8987c0f1da9e 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java @@ -20,12 +20,12 @@ import eu.stratosphere.api.java.DataSet; import eu.stratosphere.api.java.functions.CrossFunction; import eu.stratosphere.api.java.operators.translation.PlanCrossOperator; -//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator -import eu.stratosphere.api.java.tuple.*; -//CHECKSTYLE.ON: AvoidStarImport import eu.stratosphere.api.java.typeutils.TupleTypeInfo; import eu.stratosphere.api.java.typeutils.TypeExtractor; import eu.stratosphere.api.java.typeutils.TypeInformation; +//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator +import eu.stratosphere.api.java.tuple.*; +//CHECKSTYLE.ON: AvoidStarImport /** * A {@link DataSet} that is the result of a Cross transformation. @@ -71,6 +71,17 @@ protected eu.stratosphere.api.common.operators.DualInputOperator translateToD // Builder classes for incremental construction // -------------------------------------------------------------------------------------------- + /** + * A Cross transformation that wraps pairs of crossed elements into {@link Tuple2}.
+ * It also represents the {@link DataSet} that is the result of a Cross transformation. + * + * @param The type of the first input DataSet of the Cross transformation. + * @param The type of the second input DataSet of the Cross transformation. + * @param The type of the result of the Cross transformation. + * + * @see Tuple2 + * @see DataSet + */ public static final class DefaultCross extends CrossOperator> { private final DataSet input1; @@ -88,6 +99,16 @@ public DefaultCross(DataSet input1, DataSet input2) { this.input2 = input2; } + /** + * Finalizes a Cross transformation by applying a {@link CrossFunction} to each pair of crossed elements.
+ * Each CrossFunction call returns exactly one element. + * + * @param function The CrossFunction that is called for each pair of crossed elements. + * @return An CrossOperator that represents the crossed result DataSet + * + * @see CrossFunction + * @see DataSet + */ public CrossOperator with(CrossFunction function) { TypeInformation returnType = TypeExtractor.getCrossReturnTypes(function, input1.getType(), input2.getType()); return new CrossOperator(input1, input2, function, returnType); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java index 7efbbce1e3082..9a4ac248d94b3 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java @@ -592,7 +592,8 @@ public JoinOperatorSets(DataSet input1, DataSet input2, JoinHint hint) { * * @param fields The indexes of the Tuple fields of the first join DataSets that should be used as keys. * @return An incomplete Join transformation. - * Call {@link JoinOperatorSetsPredicate#equalTo(int...)} to continue the Join. + * Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or {@link JoinOperatorSetsPredicate#equalTo(KeySelector)} + * to continue the Join. * * @see Tuple * @see DataSet @@ -601,20 +602,29 @@ public JoinOperatorSetsPredicate where(int... fields) { return new JoinOperatorSetsPredicate(new Keys.FieldPositionKeys(fields, input1.getType())); } - public > JoinOperatorSetsPredicate where(KeySelector keyExtractor) { - return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys(keyExtractor, input1.getType())); + /** + * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.
+ * The KeySelector function is called for each element of the first DataSet and extracts a single + * key value on which the DataSet is joined.
+ * + * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined. + * @return An incomplete Join transformation. + * Call {@link JoinOperatorSetsPredicate#equalTo(int...)} or {@link JoinOperatorSetsPredicate#equalTo(KeySelector)} + * to continue the Join. + * + * @see KeySelector + * @see DataSet + */ + public > JoinOperatorSetsPredicate where(KeySelector keySelector) { + return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys(keySelector, input1.getType())); } -// public JoinOperatorSetsPredicate where(String keyExpression) { -// return new JoinOperatorSetsPredicate(new Keys.ExpressionKeys(keyExpression, input1.getType())); -// } - // ---------------------------------------------------------------------------------------- /** * Intermediate step of a Join transformation.
* To continue the Join transformation, select the join key of the second input {@link DataSet} by calling - * {@link JoinOperatorSetsPredicate#equalTo(int...)}. + * {@link JoinOperatorSetsPredicate#equalTo(int...)} or {@link JoinOperatorSetsPredicate#equalTo(KeySelector)}. * */ public class JoinOperatorSetsPredicate { @@ -642,7 +652,7 @@ private JoinOperatorSetsPredicate(Keys keys1) { * the element of the first input being the first field of the tuple and the element of the * second input being the second field of the tuple. * - * @param fields The indexes of the Tuple fields of the second join DataSets that should be used as keys. + * @param fields The indexes of the Tuple fields of the second join DataSet that should be used as keys. * @return A DefaultJoin that represents the joined DataSet. */ public DefaultJoin equalTo(int... fields) { @@ -658,17 +668,13 @@ public DefaultJoin equalTo(int... fields) { * the element of the first input being the first field of the tuple and the element of the * second input being the second field of the tuple. * - * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which it is joined. + * @param keySelector The KeySelector function which extracts the key values from the second DataSet on which it is joined. * @return A DefaultJoin that represents the joined DataSet. */ - public DefaultJoin equalTo(KeySelector keyExtractor) { - return createJoinOperator(new Keys.SelectorFunctionKeys(keyExtractor, input2.getType())); + public DefaultJoin equalTo(KeySelector keySelector) { + return createJoinOperator(new Keys.SelectorFunctionKeys(keySelector, input2.getType())); } -// public DefaultJoin equalTo(String keyExpression) { -// return createJoinOperator(new Keys.ExpressionKeys(keyExpression, input2.getType())); -// } - protected DefaultJoin createJoinOperator(Keys keys2) { if (keys2 == null) { throw new NullPointerException("The join keys may not be null."); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java index 3fe7ce2665393..5a81a75f2f046 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SingleInputUdfOperator.java @@ -82,6 +82,24 @@ public O withBroadcastSet(DataSet data, String name) { return returnType; } + /** + * Adds a constant-set annotation for the UDF. + * + *

+ * Constant set annotations are used by the optimizer to infer the existence of data properties (sorted, partitioned, grouped). + * In certain cases, these annotations allow the optimizer to generate a more efficient execution plan which can lead to improved performance. + * Constant set annotations can only be specified if the second input and the output type of the UDF are of {@link Tuple} data types. + * + *

+ * A constant-set annotation is a set of constant field specifications. The constant field specification String "4->3" specifies, that this UDF copies the fourth field of + * an input tuple to the third field of the output tuple. Field references are zero-indexed. + * + *

+ * NOTICE: Constant set annotations are optional, but if given need to be correct. Otherwise, the program might produce wrong results! + * + * @param constantSet A list of constant field specification Strings. + * @return This operator with an annotated constant field set. + */ public O withConstantSet(String... constantSet) { SingleInputSemanticProperties props = SemanticPropUtil.getSemanticPropsSingleFromString(constantSet, null, null, this.getInputType(), this.getResultType()); this.setSemanticProperties(props); diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java index 18303916fbde9..b9648afb2527b 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/TwoInputUdfOperator.java @@ -86,7 +86,22 @@ public O withBroadcastSet(DataSet data, String name) { } /** - * Allows to give specifications about constant sets directly in the code. Null values are allowed for not specified sets. + * Adds a constant-set annotation for the first input of the UDF. + * + *

+ * Constant set annotations are used by the optimizer to infer the existence of data properties (sorted, partitioned, grouped). + * In certain cases, these annotations allow the optimizer to generate a more efficient execution plan which can lead to improved performance. + * Constant set annotations can only be specified if the first input and the output type of the UDF are of {@link Tuple} data types. + * + *

+ * A constant-set annotation is a set of constant field specifications. The constant field specification String "4->3" specifies, that this UDF copies the fourth field of + * an input tuple to the third field of the output tuple. Field references are zero-indexed. + * + *

+ * NOTICE: Constant set annotations are optional, but if given need to be correct. Otherwise, the program might produce wrong results! + * + * @param constantSetFirst A list of constant field specification Strings for the first input. + * @return This operator with an annotated constant field set for the first input. */ @SuppressWarnings("unchecked") public O withConstantSetFirst(String... constantSetFirst) { @@ -98,6 +113,24 @@ public O withConstantSetFirst(String... constantSetFirst) { return returnType; } + /** + * Adds a constant-set annotation for the second input of the UDF. + * + *

+ * Constant set annotations are used by the optimizer to infer the existence of data properties (sorted, partitioned, grouped). + * In certain cases, these annotations allow the optimizer to generate a more efficient execution plan which can lead to improved performance. + * Constant set annotations can only be specified if the second input and the output type of the UDF are of {@link Tuple} data types. + * + *

+ * A constant-set annotation is a set of constant field specifications. The constant field specification String "4->3" specifies, that this UDF copies the fourth field of + * an input tuple to the third field of the output tuple. Field references are zero-indexed. + * + *

+ * NOTICE: Constant set annotations are optional, but if given need to be correct. Otherwise, the program might produce wrong results! + * + * @param constantSetSecond A list of constant field specification Strings for the second input. + * @return This operator with an annotated constant field set for the second input. + */ @SuppressWarnings("unchecked") public O withConstantSetSecond(String... constantSetSecond) { DualInputSemanticProperties dsp = SemanticPropUtil.getSemanticPropsDualFromString(null, constantSetSecond, From 0e3fcc90682fa1b05ad886b4f1c04b390c3b89b2 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Fri, 16 May 2014 20:38:37 +0200 Subject: [PATCH 043/182] Added UnsortedGrouping to separate Grouping methods from SortedGrouping methods --- .../src/test/resources/AvroTestProgram.jar | Bin 5706 -> 5713 bytes .../src/test/resources/test.jar | Bin 2546 -> 4929 bytes .../eu/stratosphere/api/java/DataSet.java | 52 +++++---- .../api/java/operators/Grouping.java | 84 +------------- .../api/java/operators/UnsortedGrouping.java | 107 ++++++++++++++++++ 5 files changed, 136 insertions(+), 107 deletions(-) create mode 100644 stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java diff --git a/stratosphere-addons/avro/src/test/resources/AvroTestProgram.jar b/stratosphere-addons/avro/src/test/resources/AvroTestProgram.jar index 5ac99385809ad3d3a6a1eb1cd1d8e091d43e4190..eb56b623a8628ef9843902e1d5705151e092ddda 100644 GIT binary patch delta 1210 zcmV;r1V#JGEYU0tP)h>@6aWYa2mqV0u#pXz1COz=v8en41COz=v!?@z0|Sq-u(LA< z0|Nt(v9Pl_3dsTkkFl_ks6n$v4b%YxkFl_m9S=xFvIJl;^f~6w0rM1@Z zdX~~kdz4BcR4@?aRRmu)$2MF|?lOCK415wlhyMh}bo>DR<8&NH{W6ZvZemI$X$i^9 zKDYb4{r0(ietW#%XWgmA+RLUYNvXyvuMoFTzG@ z+E8ks;g*_4v<nseYmRL$2Epa zsYX>YaGl{^i?zSGc(@)2X`&Y&sOUEt#!@GCdmM;;EcZ--2LYv4rN5QLM@TW8bEM#Z zq0pOS*f|+YA#(g;(f3p#jupXvO_}|YP|Gr~W_ie0{YW~3$~?&Mu+<$e@uHw*$sLMk zVgQ{cMqn8jW$14TK&KUSSU$#WYLp@`G2|L=PTN0?JV&2bYmt}cYLnCLkQ}nLiEZKz z?iv_p7;ViniF^2j!N^NLDm^}AxS!g8olZb1Bc690-25K zXW7Jb^<1Tsn8Oo7?nj~Si0wCZL{=NN+!C($y{-CiK8Xc8ocwaA(b3Af4`H=29a`q0*AL4l=P z+-)^~^^9NeCB3|wY_CFLIR6UM(#>#;5)9xiT0V%lw4y#o-Eiw<_q{8al%JUHwP6QO1W$BNYs69bu@9;rMfZb1N3hmN8br z1g$+N!+sF&jaM*DvzanxH;2pkY*S&1X-rpjrr#**3v}Rjn*TvW_a`!x{R{l9v5r?! z(AuKLw?zLeq3obOUnZ+I@){@oD#d(+ZmiK$9P4D;Q`c?Ob(^GX!-C-0DuYr08mQ-0u%rg00;;O z0GqL}lYSFJ1COz=lf)Bd1COz=lPwfy1COz=la~~31COz=lL!@T1COz=lXn$n4Ue&~ YL?}>9l>-C-0DzO~6@6aWYa2mo=StdR|v16ZQ0v8en416ZQ0v!?@z0|Qv1tg|x* z0|NtCqO7wy3dsTkSfZ?vs6ms>6$`Ub4aormSfZ?x9S=xd`jv7Q=8;8@I~Dq8G%`f-g%vGH{LI+W!q4pN6NQwaTMC6aBcZ z-p4x(SJI7GGH`?8UW?q{T0VRd3TdJb?<()N7{=2lPJ0}PeJu8Dk%u9LR@C24;eDhT zE;v$u@JQ&xG3=alrVu%PspNZ#h+`z!r_k&NLM_YCn&S~)_hab@qIrnnL8~WT;Uz)K zk~`$j#2~s%jKVT7#xT$nfDS7tSw6rWDwGlr81jt|r}dx4o}=^CTIRt*ZF05~l0mjM zu}yr4y9Op0##*yX;T}F>FbdL-gGYxdi?GnE$ZExwOThKMvsK+Krm%$1=uHdViA3{$RSUV~=j%;tO@wiO zF;Wef%L4W8o(BDwq03c8s)}EA{ssyO;qB9P)MA%pXm_h!;%0NpTHVEngD!%PWM_Pii4uCHqmpu2R3;cly;XZ@lt>E+dbWJeVW!^M|6E8PUgT7iN5IU=oB9k*+V8=y>s z`nAy$eVNc`(99yqNVX3hq4Nj5ZPM7G7jNSNjj5^v$%`br@ER_W)@6E5($4GJ&0l2E zPI7H1F`oGmJtR1C}?e2 z<6EJBmQZ$5ov)Hs8(B@z{yO=5h#qXvQxZ=|x3|8xS>M~Dy*7NIS;^r`l0)QwoBVx6 zcm4p$uSpJ){D$Nt$!|$=k`DSI{smA=2MG4f^CtWQ003|b002-+0Rj{N6aWYa2mo=S ztdoTkLjzc%tdq+VW&>EFtdlepX9HNGtdpJ;ZUb1NtdkEFYy()Ltdo8fW(`=PtVH(B V^CtWQ003~4@fBJIZxa9j008V$N;&`l diff --git a/stratosphere-clients/src/test/resources/test.jar b/stratosphere-clients/src/test/resources/test.jar index 53261b5a674b7f7ea338814e44d1f83993f4ab22..d0ce39b2a200e279c55f439599cce7c1bd382ba5 100644 GIT binary patch literal 4929 zcmbuD2T&7Sx`sn7 zVg--`47E(5;(A8f5JRYup0*a!RKigEYybdY>y^3`#R~+yDZr|KC*iWS%CW-u)N2Dt zi?t%!!zL+)`~}!=m@HY=$_N0hDUv=OVTVQ#R9U!NP1{q{Yn^rAXfg=4wH_m)N;kA{ zqFz-WMuO0%qRv&@k4IjNU~#b!{}{~Rp9WLEICa550?`2uXx~6Tcb5Pch)akA%Fo*c z;^`3N015UDaCY*=_@E&ew1+pu;-ZE9Zs87S2MH%{hrqxpbNxV`9?)qogU5Ae8(GSZ ztt`7RON*IahME^I9c`rfsBiei#)c4f3>I#nh7B0zRa@Fe63xt?l_ZK*)DDP>$=A@B zJ>$=Nonan%HNNWIX|4SJG%)VMJgNJ;BNtMHe$3D zQI!T6Cd;*_2=9agM4s$8W_`8+85+*>D)?lp*3U6NNK2NDuw24jzw`6TK)04` z5G-?GKFNLc+gwb1(ERwb{Kp0LZfQZ2mhfW>Gm)M5TE@9@&Z4@jwboG zDSWHnjAyffV)galoUd1>C&GAVY^t&^Y0JJU)JwXowj^0Ph3N7#!!^f>B8$4GPupds zc<^f{T)ikfy*NjRXwo=r5L2|tB@EKFwfQvD`@Sq`-5tLFGue2to6!B0w31RYbK1YN ziuGQxOiOQd5FcTQslk`e%eNb2=J5d4MC6pVy54ZIev9oAxj}Qr`)KvUKX z#!lc&bryMse44_wiD@l9l^0S4J<%@B6l!}W0wh;O^M;>1I0>g(J0u%|qdidJ_PG`l zf;rb%)@KOXiKMp@#-nvSz8aL4JPR+#Ug2-?v+y0Hzz*j?Jw#&%564nsfms~(eB-g? z?^f?KlsqcTH1LLw0FPFrdu~3xuS~qHjCkmt=>!!k$(ixgl}BXLM~Io&gr4eYOqj1x zNi_&4v#g0eZd!hdHo2y-_w@NnGp?k0L;rC{Q}VrR549=GrA1!-e)%EO_lJYv*^Ev* zBMRap{+2Z%ta?rPdrAQA=~F0gyoDq(@wGk>#%{qOebwP(W<2{z?ah!?%0%z7Zw9ow zHnmoH5BSW>q&e>O&?z?9k<@M64pVJ=cN~#L9qYjz(tc zFBH?;SiU=c3uvJibbKJ>O@gEc&N_8{Wh4MHSbvB=el1!0n0)1vrqx`vtrI`PhdeM! zP;89nE-10R>#hBB1SS|U>$T|pELzuQB5H6fkG^eIq2)VuRUciVqjiEiyIJCumqLLH zPQVfB?T61RuBvOJ!n+UV>gJO6kk91Ihhg$HJUgkLgDG348)p=utT_IVT9X{NOjoRLZxYG zvEzpiD`m6_VCEmMkXbf}4W0H{fn0a#IkQFNN&@7eRtKIy^X}dGw>Oc8l$zXqwwGt9 zXXf|qZk=d_sN7d%r(_sI#-|UH-~RcfCGKGS-cT-WV9|kINy_fj=r7lM8$^q4`Jh^| z`DEK~&D*^`K~Gzydhuw5-j6T0wTFz*3WQ!x9d5UBMyDW3e})@zik+@ZMUMON>E0y% z5F7Yhp9Jr=+o*ui%fWc#y2|*Z*YStIR(wd*yvE(EdH7xYDd{e6YjQ}uxWE!@PYKbJgQa^`?6-`)m!?3=W{QR+; zJY_LxU`HSHJ56qJ`V_y>BfsEeqjBZ}0-G}Ih2#7t8b2+yq?%iXUC|v^@lZCF6Cd+^ zxglWvWC4m_n4~zuN=#h+jychBK4P&YjIPl}KrPnw!|u12aIE|Gp9QxV*&*Z3XjB#| zi1ep8@@l3Zq#Iusesi#pCc-U`5&-a|`*$4F_|G}`4+ILD`g*zec!aqG{2hd^O|6hD zP{uRxjHBhluE$WVH!)Z*Ia<%?5b`ni3@XOK)+|MACPtwl=SSFd$LCTrwO`-pLQGB6 zj;Zbvi{e$udZlSZvs`Y;P5v=CzaKHLdb+-OYz7dyJ;7+N8@O8u zOxker@d|7*J+a`GI+k!}ar@+|t0=x5_HGRQR3H4+NyFE7nb^$H36r&ayETgUaK;Zn_Abp=8X* zrDAuO2&*4D@B<|6+i;nfm%9fPh*xGYa$9-a^H#;(n^ z9mj`T)fqeu7EEgxzsy)pJKrMDTe>yz;Pc9RxoS#a689Lr|#aYQbm@8cQ+CJoO63Cuk*^MleQe9Ofs-Ws_ufS#d z{6Z?7@L;6Nm`U8NIOtil;CObKSt%qYNISeKl7ik>mzwvc7hL2;Nve6oeAEGWHpf8B zs^Qun=4@Z-3gpMsq-v5H8*K;I+N9B{RaJQH=+ea*-R=fU#mq}Ru)(#k1bipM+UqL} zv7f%Yp*>n~xEAH+z)si4|6*EAAIKM5oQ?n}=(Z}Q>hx_N?411kh5#O~MYn2|_)=Kr#A z;Z>=q`HA(m+GGGgKGnbbv-1CkKl|U7brZ>A!Fq1XJm+ks0gJV^v8_VE$V=QCNFC*( zdiE+7O&VSp2GG#u%{R3!7Ee#l&IWh<)GxoPykyan(_hbX9(AU0N|j$2&O(|MHRb`X z`-vH8?$~cy&F?5YJ0sT+zIW@wsU>lIDcs>|Us`r+8eFq0G->0yysRZ&BL5`eo>RpT=>r&&RY7e*8 zgwE7%p+to7tTbwJ$AubhBMgeB5$@`;cOOGMumvW=2*WzG&8)$F$r(Mdrot%8bevZr zCN$6@g|(DF3QQl4&?z|qO**Y{&pxXlm(G?k$SYO~LzU8!5@tKHS3(!i-o~B!KtnIn zOqR_05UVaRcRukw+~%%Smt4TyYIXS+-b*55pxN`}k^Vd(=hFu&wv5q7TcMMfZ``jU z=;L-pVftCe@9`kKj*Y@vYgxAs$-W%@izRw#<##;7KYxJQQ@;vIfaZ(q#eE=jTjpZ} zV-3i1Zcy+_6YA6F>cnfgruD@z=#7}OHafX(|0HNmBQWwqzs!(N@f5n#NdHwY)y=PD z%gI%;fqS>TTiFQXdz7G8EW&L0k0s}nUf~q{6QkU0%O7^sbo+tBP<}R&VDC}?h{EP; zidsXsUG^5%{9?ezjQM_7+d?^P?e4x;pfOA#Y%KyTi~|{Y-#c|ZamLn7ylHb({kq_XKAL zzzOB~J${G3n&W6=hs4N>e|?m$&9D;}DlZfxxCtEIVcznI@ki2j@YXIg9R``FcKEoB z9fW4JqRX1<)z&YsD-Cclzowq9r=%#BRIH^X6p75f6B=DrqL-13ZLodh;Hy`=-rWah z9LLP{n>Y_ib%Rv|23v^@fORI^^uB3J_>U_VNA>l$Qe#Tr+^NEjCQd%=!z2oU2<%K- zRqXodqELeOQmKZ-@&jkS^(J$o>I8)^sZ3F7MOERgylO9ZPYRg#1Z)oLGtveU7?kf~fd2lCI z*`L#10`k~)rY<(BGrr#Ae94ceJU4woPFkwPZP&YQ&*Y|)WpXaVzcr`}KN8@0z?t8Y zf(U2oLh1v$~>Qwp)yDMYP47kSjXRZ90cMI#3CSP9e)K*A6DxYJ!5B6Y83@Di-dn zR^P9kz@c5gYuFmn@ex+CAKnAcpESeOZfSoKoHx;7L~6~2YWSRNKq<6ilDcmQu1O5< zRJ9WHtn1N+famw0LE+V{%8HG)aI|Z{0>$AYBPU_Dt}ha#rFRhvHY{xU-_ug!op%9-eIb^!n)OohEy4ub(Na<8BM!g0{SHq#*p zPtymR>&E>E3u>6vX9gVY!xz9(s)D(e2jayEVuG80P&W!BVrW5DK)4ci+Rc%>fSPTzvO`#^scAJRH_Qn2_9`4rP=^1Um;(`zaVxSArkKpfx z3&bICAugBveQ}73E+iL3uwS6Nn_m!~h&aEEP&)|a=j)4eBYOMck6HTp;PBpLT%eqr zuM2@tYBdrpq$j*(qD|@|!5<{+(gry7ZWUF~jQXI~m@ETjnl1>E)M(65cCaB4NS^sW zO17>gX|GLIAzG6N>e@?8Np|Xo9%!0HB`73oirJJ|ckg-VA~Q8|GX(4`p=KQK+*8;)S4^%R7F}&9qb9%2sIJV;-z4)s6fdi2ec%i|si@!_rX-WS_B9v<`Ee3ean{t2bm_&X5ebj)nfkunF zBv7qj`E^90c#-3WoHL0ILJ;=4+0W{ANaq4P-jcgVk2UM^P>OH5-fOGscZd-Q z*nRwmVmvL)YwV~2d*5V)&BuoXW2~P2xppBKNSY=&3R&bzvS#@?jj&V~Zc$mqp%>Tb zn+2;uE1i%i+adhRvW1#tHi06!b?T4qj<|zLwD~5Fl2x@^f*N1QnX;!rgOv+25gIA- zi0P+A(KY_>=pxvvBEK=1?HZ?(o!2Cck;#gUaeGTY@T@<*%nW5yn<&bdd6{CTo+t#F z2g7+3*x)64kjR&)edSctm~yXu=06NXUlW6wq(=u zvJ384PY$Jl{N3}?i7Bla?SoJU?r435{6WO`(9u~gt@@hF4H?oXIUO|SMhu<5iK;`^&x1uA3g75?42jj6rBGmC0^Uo7Yn^r+REafpHDU(EEeK;2|kH_~*ka z>7zOxE5okPdzTxNO%UQ5M>96|KL~?*G7;a9V>*AFi0|FxWzQpfe{ydEQP0^}(AdFG z86(ue%TUs$Y8_AdY@dCz_ZzJ4Axukoo&XOqH|K7fR!8bFYKfe{X`OnvW%9E)Ll6vo zxe1+9Z86VNRwvW%tPm_nmF5+~RVwS;FC6z1=pE6P@BliHM&9 zMW)^}2fzC?qj)v2jK52yG97xcCvEeZMnlwuyA73W9@>|wE_ohB5SyI&=z7-cjzQ)0 zELWKL>p0=G^!sWDtRSe({X6ofG!Fo7Z~*|?!2cxwGyjwP{}O$!m5;p;S~%iF4d1-= zrIL`@OFpH1Qh1K1rTI2|&ALKfBS4c4WnPrc@Ta1xD&ZgHgZxiS5fPtmY(CUJRmFSQ zXdg4xbDCM#w3wrzw6qu*iR6)wmNdy>T#w222ZyD%yGvMHHl7ZECk@$!x4)Q%8h>8e z{Y*JP?12Dt94=Ha3|q+bbc(kA2VX|4&Dyw%A4+=m3D`HEYrU27GQ{)zAf(C6$_so3yu z5~pmg=PuxnEHtWiGapZR$~;Btk7M~o<`1w|g6EQdBD_R7j}Ph!7ILe=hh+QSs>Fbq zeZzci?**6cI(D_9RA-BN++Bplz?LVO>?4f#8DGUKS6<=GLGU!vKA8OqmFk^!lT?Tt zqZTYVcjI{VrkWC+*fm zt+-V7U~E*ssH+oF$Vk-yBef+uIMy89~8o=4D z6W>|v!;d_tn5!exJhAMQ1G{ATgrC(tlN6N^DKkr+W>MrtjSN14B z&9$zg;8ajR!1G{vAiSqaCYNxa|nl|8N8jPGP{W z#PWMs*hwsZuYZ?cu$W(6c7nz4p0g7ye$~Bg`2UL;|2p^g&CdK9x{dB`ukzo|{mb95 WnBCkvO}VzuNe%#@V*CF9fIk5Plw8mN diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java index aadef1b19c96e..190ea6fe73eb0 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java @@ -19,6 +19,7 @@ import eu.stratosphere.api.common.io.FileOutputFormat; import eu.stratosphere.api.common.io.OutputFormat; import eu.stratosphere.api.java.aggregation.Aggregations; +import eu.stratosphere.api.java.functions.CoGroupFunction; import eu.stratosphere.api.java.functions.FilterFunction; import eu.stratosphere.api.java.functions.FlatMapFunction; import eu.stratosphere.api.java.functions.GroupReduceFunction; @@ -31,21 +32,26 @@ import eu.stratosphere.api.java.operators.AggregateOperator; import eu.stratosphere.api.java.operators.CoGroupOperator; import eu.stratosphere.api.java.operators.CoGroupOperator.CoGroupOperatorSets; -import eu.stratosphere.api.java.operators.CrossOperator.DefaultCross; import eu.stratosphere.api.java.operators.CrossOperator; +import eu.stratosphere.api.java.operators.CrossOperator.DefaultCross; import eu.stratosphere.api.java.operators.CustomUnaryOperation; import eu.stratosphere.api.java.operators.DataSink; import eu.stratosphere.api.java.operators.FilterOperator; import eu.stratosphere.api.java.operators.FlatMapOperator; import eu.stratosphere.api.java.operators.Grouping; +import eu.stratosphere.api.java.operators.JoinOperator; import eu.stratosphere.api.java.operators.JoinOperator.JoinHint; import eu.stratosphere.api.java.operators.JoinOperator.JoinOperatorSets; import eu.stratosphere.api.java.operators.Keys; import eu.stratosphere.api.java.operators.MapOperator; +import eu.stratosphere.api.java.operators.ProjectOperator; import eu.stratosphere.api.java.operators.ProjectOperator.Projection; import eu.stratosphere.api.java.operators.ReduceGroupOperator; import eu.stratosphere.api.java.operators.ReduceOperator; +import eu.stratosphere.api.java.operators.SortedGrouping; import eu.stratosphere.api.java.operators.UnionOperator; +import eu.stratosphere.api.java.operators.UnsortedGrouping; +import eu.stratosphere.api.java.record.functions.CrossFunction; import eu.stratosphere.api.java.tuple.Tuple; import eu.stratosphere.api.java.tuple.Tuple2; import eu.stratosphere.api.java.typeutils.InputTypeConfigurable; @@ -251,10 +257,6 @@ public ReduceGroupOperator reduceGroup(GroupReduceFunction reduc // return new DistinctOperator(this, new Keys.SelectorFunctionKeys(keyExtractor, getType())); // } -// public DistinctOperator distinct(String fieldExpression) { -// return new DistinctOperator(this, new Keys.ExpressionKeys(fieldExpression, getType())); -// } - // public DistinctOperator distinct(int... fields) { // return new DistinctOperator(this, new Keys.FieldPositionKeys(fields, getType(), true)); // } @@ -267,42 +269,42 @@ public ReduceGroupOperator reduceGroup(GroupReduceFunction reduc * Groups a {@link DataSet} using a {@link KeySelector} function. * The KeySelector function is called for each element of the DataSet and extracts a single * key value on which the DataSet is grouped.
- * This method returns a {@link Grouping} on which one of the following grouping transformation - * needs to be applied to obtain a transformed DataSet. + * This method returns an {@link UnsortedGrouping} on which one of the following grouping transformation + * can be applied. *

    - *
  • {@link Grouping#aggregate(Aggregations, int)} - *
  • {@link Grouping#reduce(ReduceFunction)} - *
  • {@link Grouping#reduceGroup(GroupReduceFunction)} + *
  • {@link UnsortedGrouping#sortGroup(int, eu.stratosphere.api.common.operators.Order)} to get a {@link SortedGrouping}. + *
  • {@link Grouping#aggregate(Aggregations, int)} to apply an Aggregate transformation. + *
  • {@link Grouping#reduce(ReduceFunction)} to apply a Reduce transformation. + *
  • {@link Grouping#reduceGroup(GroupReduceFunction)} to apply a GroupReduce transformation. *
* * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which it is grouped. - * @return A Grouping on which a transformation needs to be applied to obtain a transformed DataSet. + * @return An UnsortedGrouping on which a transformation needs to be applied to obtain a transformed DataSet. * * @see KeySelector * @see Grouping + * @see UnsortedGrouping + * @see SortedGrouping * @see AggregateOperator * @see ReduceOperator * @see GroupReduceOperator * @see DataSet */ - public > Grouping groupBy(KeySelector keyExtractor) { - return new Grouping(this, new Keys.SelectorFunctionKeys(keyExtractor, getType())); + public > UnsortedGrouping groupBy(KeySelector keyExtractor) { + return new UnsortedGrouping(this, new Keys.SelectorFunctionKeys(keyExtractor, getType())); } -// public Grouping groupBy(String fieldExpression) { -// return new Grouping(this, new Keys.ExpressionKeys(fieldExpression, getType())); -// } - /** * Groups a {@link Tuple} {@link DataSet} using field position keys.
* Note: Field position keys only be specified for Tuple DataSets.
* The field position keys specify the fields of Tuples on which the DataSet is grouped. - * This method returns a {@link Grouping} on which one of the following grouping transformation - * needs to be applied to obtain a transformed DataSet. + * This method returns an {@link UnsortedGrouping} on which one of the following grouping transformation + * can be applied. *
    - *
  • {@link Grouping#aggregate(Aggregations, int)} - *
  • {@link Grouping#reduce(ReduceFunction)} - *
  • {@link Grouping#reduceGroup(GroupReduceFunction)} + *
  • {@link UnsortedGrouping#sortGroup(int, eu.stratosphere.api.common.operators.Order)} to get a {@link SortedGrouping}. + *
  • {@link Grouping#aggregate(Aggregations, int)} to apply an Aggregate transformation. + *
  • {@link Grouping#reduce(ReduceFunction)} to apply a Reduce transformation. + *
  • {@link Grouping#reduceGroup(GroupReduceFunction)} to apply a GroupReduce transformation. *
* * @param fields One or more field positions on which the DataSet will be grouped. @@ -310,13 +312,15 @@ public > Grouping groupBy(KeySelector keyExtrac * * @see Tuple * @see Grouping + * @see UnsortedGrouping + * @see SortedGrouping * @see AggregateOperator * @see ReduceOperator * @see GroupReduceOperator * @see DataSet */ - public Grouping groupBy(int... fields) { - return new Grouping(this, new Keys.FieldPositionKeys(fields, getType(), false)); + public UnsortedGrouping groupBy(int... fields) { + return new UnsortedGrouping(this, new Keys.FieldPositionKeys(fields, getType(), false)); } // -------------------------------------------------------------------------------------------- diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java index 280f88b018e1a..25a0ed18479ee 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Grouping.java @@ -15,11 +15,7 @@ package eu.stratosphere.api.java.operators; import eu.stratosphere.api.common.InvalidProgramException; -import eu.stratosphere.api.common.operators.Order; import eu.stratosphere.api.java.DataSet; -import eu.stratosphere.api.java.aggregation.Aggregations; -import eu.stratosphere.api.java.functions.GroupReduceFunction; -import eu.stratosphere.api.java.functions.ReduceFunction; /** * Grouping is an intermediate step for a transformation on a grouped DataSet.
@@ -34,7 +30,7 @@ * * @see DataSet */ -public class Grouping { +public abstract class Grouping { protected final DataSet dataSet; @@ -61,83 +57,5 @@ protected DataSet getDataSet() { protected Keys getKeys() { return this.keys; } - - // -------------------------------------------------------------------------------------------- - // Operations / Transformations - // -------------------------------------------------------------------------------------------- - - /** - * Applies an Aggregate transformation on a grouped {@link Tuple} {@link DataSet}.
- * Note: Only Tuple DataSets can be aggregated. - * The transformation applies a built-in {@link Aggregations Aggregation} on a specified field - * of a Tuple group. Additional aggregation functions can be added to the resulting - * {@link AggregateOperator} by calling {@link AggregateOperator#and(Aggregations, int)}. - * - * @param agg The built-in aggregation function that is computed. - * @param field The index of the Tuple field on which the aggregation function is applied. - * @return An AggregateOperator that represents the aggregated DataSet. - * - * @see Tuple - * @see Aggregations - * @see AggregateOperator - * @see DataSet - */ - public AggregateOperator aggregate(Aggregations agg, int field) { - return new AggregateOperator(this, agg, field); - } - - /** - * Applies a Reduce transformation on a grouped {@link DataSet}.
- * For each group, the transformation consecutively calls a {@link ReduceFunction} - * until only a single element for each group remains. - * A ReduceFunction combines two elements into one new element of the same type. - * - * @param reducer The ReduceFunction that is applied on each group of the DataSet. - * @return A ReduceOperator that represents the reduced DataSet. - * - * @see ReduceFunction - * @see ReduceOperator - * @see DataSet - */ - public ReduceOperator reduce(ReduceFunction reducer) { - return new ReduceOperator(this, reducer); - } - - /** - * Applies a GroupReduce transformation on a grouped {@link DataSet}.
- * The transformation calls a {@link GroupReduceFunction} for each group of the DataSet. - * A GroupReduceFunction can iterate over all elements of a group and emit any - * number of output elements including none. - * - * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. - * @return A GroupReduceOperator that represents the reduced DataSet. - * - * @see GroupReduceFunction - * @see GroupReduceOperator - * @see DataSet - */ - public ReduceGroupOperator reduceGroup(GroupReduceFunction reducer) { - return new ReduceGroupOperator(this, reducer); - } - - // -------------------------------------------------------------------------------------------- - // Group Operations - // -------------------------------------------------------------------------------------------- - - /** - * Sorts {@link Tuple} elements within a group on the specified field in the specified {@link Order}.
- * Note: Only groups of Tuple elements can be sorted.
- * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. - * - * @param field The Tuple field on which the group is sorted. - * @param order The Order in which the specified Tuple field is sorted. - * @return A SortedGrouping with specified order of group element. - * - * @see Tuple - * @see Order - */ - public SortedGrouping sortGroup(int field, Order order) { - return new SortedGrouping(this.dataSet, this.keys, field, order); - } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java new file mode 100644 index 0000000000000..711bc2498f89d --- /dev/null +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java @@ -0,0 +1,107 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.java.operators; + +import eu.stratosphere.api.common.operators.Order; +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.aggregation.Aggregations; +import eu.stratosphere.api.java.functions.GroupReduceFunction; +import eu.stratosphere.api.java.functions.ReduceFunction; + +public class UnsortedGrouping extends Grouping { + + public UnsortedGrouping(DataSet set, Keys keys) { + super(set, keys); + } + + // -------------------------------------------------------------------------------------------- + // Operations / Transformations + // -------------------------------------------------------------------------------------------- + + /** + * Applies an Aggregate transformation on a grouped {@link Tuple} {@link DataSet}.
+ * Note: Only Tuple DataSets can be aggregated. + * The transformation applies a built-in {@link Aggregations Aggregation} on a specified field + * of a Tuple group. Additional aggregation functions can be added to the resulting + * {@link AggregateOperator} by calling {@link AggregateOperator#and(Aggregations, int)}. + * + * @param agg The built-in aggregation function that is computed. + * @param field The index of the Tuple field on which the aggregation function is applied. + * @return An AggregateOperator that represents the aggregated DataSet. + * + * @see Tuple + * @see Aggregations + * @see AggregateOperator + * @see DataSet + */ + public AggregateOperator aggregate(Aggregations agg, int field) { + return new AggregateOperator(this, agg, field); + } + + /** + * Applies a Reduce transformation on a grouped {@link DataSet}.
+ * For each group, the transformation consecutively calls a {@link ReduceFunction} + * until only a single element for each group remains. + * A ReduceFunction combines two elements into one new element of the same type. + * + * @param reducer The ReduceFunction that is applied on each group of the DataSet. + * @return A ReduceOperator that represents the reduced DataSet. + * + * @see ReduceFunction + * @see ReduceOperator + * @see DataSet + */ + public ReduceOperator reduce(ReduceFunction reducer) { + return new ReduceOperator(this, reducer); + } + + /** + * Applies a GroupReduce transformation on a grouped {@link DataSet}.
+ * The transformation calls a {@link GroupReduceFunction} for each group of the DataSet. + * A GroupReduceFunction can iterate over all elements of a group and emit any + * number of output elements including none. + * + * @param reducer The GroupReduceFunction that is applied on each group of the DataSet. + * @return A GroupReduceOperator that represents the reduced DataSet. + * + * @see GroupReduceFunction + * @see GroupReduceOperator + * @see DataSet + */ + public ReduceGroupOperator reduceGroup(GroupReduceFunction reducer) { + return new ReduceGroupOperator(this, reducer); + } + + // -------------------------------------------------------------------------------------------- + // Group Operations + // -------------------------------------------------------------------------------------------- + + /** + * Sorts {@link Tuple} elements within a group on the specified field in the specified {@link Order}.
+ * Note: Only groups of Tuple elements can be sorted.
+ * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls. + * + * @param field The Tuple field on which the group is sorted. + * @param order The Order in which the specified Tuple field is sorted. + * @return A SortedGrouping with specified order of group element. + * + * @see Tuple + * @see Order + */ + public SortedGrouping sortGroup(int field, Order order) { + return new SortedGrouping(this.dataSet, this.keys, field, order); + } + +} From b3f3a6498f7974bab83a7e8d6816665009ebb2f7 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 15 May 2014 13:51:08 +0200 Subject: [PATCH 044/182] various fixes --- .../compiler/contextcheck/ContextChecker.java | 5 ++++- .../examples/scala/wordcount/WordCount.scala | 4 ++++ .../eu/stratosphere/nephele/jobmanager/JobManager.java | 9 ++++++--- .../eu/stratosphere/nephele/taskmanager/TaskManager.java | 3 ++- 4 files changed, 16 insertions(+), 5 deletions(-) diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/contextcheck/ContextChecker.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/contextcheck/ContextChecker.java index b6b7536848d68..91775a51ad55e 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/contextcheck/ContextChecker.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/contextcheck/ContextChecker.java @@ -16,6 +16,8 @@ import java.util.HashSet; import java.util.Set; +import com.google.common.base.Preconditions; + import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.Plan; import eu.stratosphere.api.common.operators.BulkIteration; @@ -52,9 +54,10 @@ public ContextChecker() {} * thrown. * * @param plan - * The PACT plan to check. + * The plan to check. */ public void check(Plan plan) { + Preconditions.checkNotNull(plan, "The passed plan is null."); this.visitedNodes.clear(); plan.accept(this); } diff --git a/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/wordcount/WordCount.scala b/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/wordcount/WordCount.scala index d77507d360734..e3083aa3030aa 100644 --- a/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/wordcount/WordCount.scala +++ b/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/wordcount/WordCount.scala @@ -44,6 +44,10 @@ class WordCount extends Program with ProgramDescription with Serializable { "Parameters: " } override def getPlan(args: String*) = { + if (args.size < 3) { + println(getDescription) + System.exit(1); + } getScalaPlan(args(0).toInt, args(1), args(2)) } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index d256d5176be04..f8cbbe1e161f5 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -359,9 +359,12 @@ public static void main(String[] args) { // Clean up is triggered through a shutdown hook // freeze this thread to keep the JVM alive (the job manager threads are daemon threads) - try { - new Object().wait(); - } catch (InterruptedException e) {} + Object w = new Object(); + synchronized (w) { + try { + w.wait(); + } catch (InterruptedException e) {} + } } @SuppressWarnings("static-access") diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 4b832638063c7..1ea5b1be626c4 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -556,8 +556,9 @@ public static boolean tryToConnect(InetAddress fromAddress, SocketAddress toSock socket.bind(bindP); socket.connect(toSocket, timeout); } catch (Exception ex) { + LOG.info("Failed to determine own IP address from '" + fromAddress + "': " + ex.getMessage()); if (LOG.isDebugEnabled()) { - LOG.debug("Failed on this address: " + ex.getMessage()); + LOG.debug("Failed with exception", ex); } connectable = false; } finally { From f8f2b162a4c2e3475cd7525d556ce611563edcb3 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 16 May 2014 13:03:33 +0200 Subject: [PATCH 045/182] Substract only 15% from TM heap for yarn, capped at 500MB --- .../main/java/eu/stratosphere/yarn/ApplicationMaster.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/stratosphere-addons/yarn/src/main/java/eu/stratosphere/yarn/ApplicationMaster.java b/stratosphere-addons/yarn/src/main/java/eu/stratosphere/yarn/ApplicationMaster.java index cc397c2e6b6ef..65ae114b35a51 100644 --- a/stratosphere-addons/yarn/src/main/java/eu/stratosphere/yarn/ApplicationMaster.java +++ b/stratosphere-addons/yarn/src/main/java/eu/stratosphere/yarn/ApplicationMaster.java @@ -62,6 +62,7 @@ public class ApplicationMaster { private static final Log LOG = LogFactory.getLog(ApplicationMaster.class); + private static final int HEAP_LIMIT_CAP = 500; private void run() throws Exception { //Utils.logFilesInCurrentDirectory(LOG); @@ -82,7 +83,10 @@ private void run() throws Exception { final int memoryPerTaskManager = Integer.valueOf(envs.get(Client.ENV_TM_MEMORY)); final int coresPerTaskManager = Integer.valueOf(envs.get(Client.ENV_TM_CORES)); - final int heapLimit = (int)((float)memoryPerTaskManager*0.7); + int heapLimit = (int)((float)memoryPerTaskManager*0.85); + if( (memoryPerTaskManager - heapLimit) > HEAP_LIMIT_CAP) { + heapLimit = memoryPerTaskManager-HEAP_LIMIT_CAP; + } if(currDir == null) { throw new RuntimeException("Current directory unknown"); From e28d213a3a5da2e93adf30a19a267e78aa1a88d6 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Mon, 19 May 2014 01:42:31 +0200 Subject: [PATCH 046/182] Fix NullPointer exceptions in SemanticPropsUtil --- .../api/java/functions/SemanticPropUtil.java | 50 +++++++++++++------ 1 file changed, 36 insertions(+), 14 deletions(-) diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java index 1eb0ec061fbf6..20be4821830c0 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/functions/SemanticPropUtil.java @@ -80,7 +80,9 @@ private static void parseConstantFields(String[] cf, SingleInputSemanticProperti return; } for (String s : cf) { - readConstantSet(sm, s, inType, outType, 0); + if (s != null) { + readConstantSet(sm, s, inType, outType, 0); + } } } @@ -150,7 +152,9 @@ private static void parseConstantFieldsFirst(String[] cff, DualInputSemanticProp } for (String s : cff) { - readConstantSet(dm, s, inType, outType, 0); + if (s != null) { + readConstantSet(dm, s, inType, outType, 0); + } } } @@ -160,7 +164,9 @@ private static void parseConstantFieldsSecond(String[] cfs, DualInputSemanticPro } for (String s : cfs) { - readConstantSet(dm, s, inType, outType, 1); + if (s != null) { + readConstantSet(dm, s, inType, outType, 1); + } } } @@ -170,6 +176,10 @@ private static void parseConstantFieldsFirstExcept(String[] cffe, DualInputSeman } for (String str : cffe) { + if (str == null) { + continue; + } + FieldSet fs = readFieldSetFromString(str, inType, outType); for (int i = 0; i < outType.getArity(); i++) { @@ -186,6 +196,10 @@ private static void parseConstantFieldsSecondExcept(String[] cfse, DualInputSema } for (String str : cfse) { + if (str == null) { + continue; + } + FieldSet fs = readFieldSetFromString(str, inType, outType); for (int i = 0; i < outType.getArity(); i++) { @@ -202,8 +216,10 @@ private static void parseReadFieldsFirst(String[] rf, DualInputSemanticPropertie } for (String str : rf) { - FieldSet fs = readFieldSetFromString(str, inType, outType); - dm.addReadFields1(fs); + if (str != null) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + dm.addReadFields1(fs); + } } } @@ -213,8 +229,10 @@ private static void parseReadFieldsSecond(String[] rf, DualInputSemanticProperti } for (String str : rf) { - FieldSet fs = readFieldSetFromString(str, inType, outType); - dm.addReadFields2(fs); + if (str != null) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + dm.addReadFields2(fs); + } } } @@ -229,11 +247,13 @@ private static void parseConstantFieldsExcept(String[] cfe, SingleInputSemanticP } for (String str : cfe) { - FieldSet fs = readFieldSetFromString(str, inType, outType); - - for (int i = 0; i < outType.getArity(); i++) { - if (!fs.contains(i)) { - sm.addForwardedField(i, i); + if (str != null) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + + for (int i = 0; i < outType.getArity(); i++) { + if (!fs.contains(i)) { + sm.addForwardedField(i, i); + } } } } @@ -265,8 +285,10 @@ private static void parseReadFields(String[] rf, SingleInputSemanticProperties s } for (String str : rf) { - FieldSet fs = readFieldSetFromString(str, inType, outType); - sm.addReadFields(fs); + if (str != null) { + FieldSet fs = readFieldSetFromString(str, inType, outType); + sm.addReadFields(fs); + } } } From ee54848587787b87dce8e8d3d96dc5110d9c7c11 Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Mon, 19 May 2014 02:03:22 +0200 Subject: [PATCH 047/182] Improved error messages and exceptions on non-serializable objects. --- .../NonSerializableUserCodeException.java | 39 +++++++++++++++++++ .../operators/util/UserCodeObjectWrapper.java | 8 +++- 2 files changed, 46 insertions(+), 1 deletion(-) create mode 100644 stratosphere-core/src/main/java/eu/stratosphere/api/common/NonSerializableUserCodeException.java diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/NonSerializableUserCodeException.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/NonSerializableUserCodeException.java new file mode 100644 index 0000000000000..6d227de50965a --- /dev/null +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/NonSerializableUserCodeException.java @@ -0,0 +1,39 @@ +/*********************************************************************************************************************** + * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) + * + * Licensed 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 eu.stratosphere.api.common; + +/** + * An special case of the {@link InvalidProgramException}, indicating that a part of the program + * that needs to be serializable (for shipping) is not serializable. + */ +public class NonSerializableUserCodeException extends InvalidProgramException { + + private static final long serialVersionUID = 2L; + + /** + * Creates a new exception with no message. + */ + public NonSerializableUserCodeException() { + super(); + } + + /** + * Creates a new exception with the given message. + * + * @param message The exception message. + */ + public NonSerializableUserCodeException(String message) { + super(message); + } +} diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/UserCodeObjectWrapper.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/UserCodeObjectWrapper.java index 059ceae4a0a73..e8bd38ab79dca 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/UserCodeObjectWrapper.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/UserCodeObjectWrapper.java @@ -20,6 +20,8 @@ import com.google.common.base.Preconditions; +import eu.stratosphere.api.common.NonSerializableUserCodeException; + /** * This holds an actual object containing user defined code. */ @@ -73,7 +75,7 @@ public UserCodeObjectWrapper(T userCodeObject) { Object fieldContents = f.get(current); if (fieldContents != null && !(fieldContents instanceof Serializable)) { - throw new RuntimeException("User-defined object " + userCodeObject + " (" + + throw new NonSerializableUserCodeException("User-defined object " + userCodeObject + " (" + userCodeObject.getClass().getName() + ") contains non-serializable field " + f.getName() + " = " + f.get(current)); } @@ -81,6 +83,10 @@ public UserCodeObjectWrapper(T userCodeObject) { current = newCurrent; } } + catch (NonSerializableUserCodeException e) { + // forward those + throw e; + } catch (Exception e) { // should never happen, since we make the fields accessible. // anyways, do not swallow the exception, but report it From 6b24442fce51c0e0a97421a26499a4293b85889b Mon Sep 17 00:00:00 2001 From: StephanEwen Date: Mon, 19 May 2014 02:57:19 +0200 Subject: [PATCH 048/182] Add workset iteration translation tests. Add check that solution set is joined with on correct keys. --- .../compiler/costs/DefaultCostEstimator.java | 2 +- .../plandump/PlanJSONDumpGenerator.java | 38 ++- .../compiler/AdditionalOperatorsTest.java | 1 + .../compiler/BranchingPlansCompilerTest.java | 3 +- .../pact/compiler/CompilerTestBase.java | 14 +- .../pact/compiler/DOPChangeTest.java | 1 + .../pact/compiler/GroupOrderTest.java | 1 + .../compiler/HardPlansCompilationTest.java | 1 + .../pact/compiler/ReduceAllTest.java | 1 + .../UnionPropertyPropagationTest.java | 1 + .../WorksetIterationsJavaApiCompilerTest.java | 300 ++++++++++++++++++ ...rksetIterationsRecordApiCompilerTest.java} | 18 +- .../api/common/operators/util/FieldList.java | 2 +- .../stratosphere/api/java/DeltaIteration.java | 18 +- .../api/java/operators/JoinOperator.java | 22 ++ 15 files changed, 394 insertions(+), 29 deletions(-) create mode 100644 stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsJavaApiCompilerTest.java rename stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/{WorksetIterationsCompilerTest.java => WorksetIterationsRecordApiCompilerTest.java} (94%) diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java index 034b7b459918d..058af1a682c09 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/costs/DefaultCostEstimator.java @@ -102,7 +102,7 @@ public void addBroadcastCost(EstimateProvider estimates, int replicationFactor, } else { costs.addNetworkCost(replicationFactor * estOutShipSize); } - costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * replicationFactor); + costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * replicationFactor * 100); } // -------------------------------------------------------------------------------------------- diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java index 953bfc4cf306e..d966488f33482 100644 --- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java +++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plandump/PlanJSONDumpGenerator.java @@ -395,34 +395,51 @@ private void visit(DumpableNode node, PrintWriter writer, boolean first) { if (p.getDriverStrategy() != null) { switch (p.getDriverStrategy()) { case NONE: + case BINARY_NO_OP: + break; + case UNARY_NO_OP: + locString = "No-Op"; break; + case COLLECTOR_MAP: case MAP: case FLAT_MAP: locString = "Map"; break; - case SORTED_GROUP_COMBINE: - locString = "Sorted Combine"; - break; - case SORTED_GROUP_REDUCE: - locString = "Sorted Group Reduce"; + + case ALL_REDUCE: + locString = "Reduce All"; break; - case ALL_GROUP_REDUCE: + + case ALL_GROUP_REDUCE: + case ALL_GROUP_COMBINE: locString = "Group Reduce All"; break; + case SORTED_REDUCE: - locString = "Sorted Incremental Reduce"; + locString = "Sorted Reduce"; break; - case ALL_REDUCE: - locString = "Incremental Reduce All"; + + case SORTED_PARTIAL_REDUCE: + locString = "Sorted Combine/Reduce"; break; + + case SORTED_GROUP_REDUCE: + locString = "Sorted Group Reduce"; + break; + + case SORTED_GROUP_COMBINE: + locString = "Sorted Combine"; + break; + case HYBRIDHASH_BUILD_FIRST: locString = "Hybrid Hash (build: " + child1name + ")"; break; case HYBRIDHASH_BUILD_SECOND: locString = "Hybrid Hash (build: " + child2name + ")"; break; + case NESTEDLOOP_BLOCKED_OUTER_FIRST: locString = "Nested Loops (Blocked Outer: " + child1name + ")"; break; @@ -435,12 +452,15 @@ private void visit(DumpableNode node, PrintWriter writer, boolean first) { case NESTEDLOOP_STREAMED_OUTER_SECOND: locString = "Nested Loops (Streamed Outer: " + child2name + ")"; break; + case MERGE: locString = "Merge"; break; + case CO_GROUP: locString = "Co-Group"; break; + default: throw new CompilerException("Unknown local strategy '" + p.getDriverStrategy().name() + "' in JSON generator."); diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/AdditionalOperatorsTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/AdditionalOperatorsTest.java index 19e8981de067c..e231c4dc6135b 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/AdditionalOperatorsTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/AdditionalOperatorsTest.java @@ -40,6 +40,7 @@ * Tests that validate optimizer choices when using operators that are requesting certain specific execution * strategies. */ +@SuppressWarnings("serial") public class AdditionalOperatorsTest extends CompilerTestBase { @Test diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java index 2d082f3aa7504..000f1eecbf1a6 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/BranchingPlansCompilerTest.java @@ -48,8 +48,7 @@ import eu.stratosphere.types.IntValue; import eu.stratosphere.types.LongValue; -/** - */ +@SuppressWarnings("serial") public class BranchingPlansCompilerTest extends CompilerTestBase { diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java index 4b487086bef7d..1ba9de123c3b8 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/CompilerTestBase.java @@ -50,7 +50,9 @@ /** * */ -public abstract class CompilerTestBase { +public abstract class CompilerTestBase implements java.io.Serializable { + + private static final long serialVersionUID = 1L; protected static final String IN_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random"; @@ -64,15 +66,15 @@ public abstract class CompilerTestBase { // ------------------------------------------------------------------------ - protected DataStatistics dataStats; + protected transient DataStatistics dataStats; - protected PactCompiler withStatsCompiler; + protected transient PactCompiler withStatsCompiler; - protected PactCompiler noStatsCompiler; + protected transient PactCompiler noStatsCompiler; - protected InstanceTypeDescription instanceType; + protected transient InstanceTypeDescription instanceType; - private int statCounter; + private transient int statCounter; // ------------------------------------------------------------------------ diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java index 1baa05b089f62..e387c0c8d2ef0 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/DOPChangeTest.java @@ -47,6 +47,7 @@ * parallelism between tasks is increased or decreased. * */ +@SuppressWarnings("serial") public class DOPChangeTest extends CompilerTestBase { /** diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupOrderTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupOrderTest.java index 99f9234bb224c..375c6b66abcb4 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupOrderTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/GroupOrderTest.java @@ -48,6 +48,7 @@ * This test case has been created to validate that correct strategies are used if orders within groups are * requested. */ +@SuppressWarnings("serial") public class GroupOrderTest extends CompilerTestBase { @Test diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/HardPlansCompilationTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/HardPlansCompilationTest.java index 7bf2eef9ef403..8c47aced8037c 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/HardPlansCompilationTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/HardPlansCompilationTest.java @@ -36,6 +36,7 @@ *
  • Ticket 158 * */ +@SuppressWarnings("serial") public class HardPlansCompilationTest extends CompilerTestBase { diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/ReduceAllTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/ReduceAllTest.java index b2d53ed5ee34f..52f0fa31d8253 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/ReduceAllTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/ReduceAllTest.java @@ -32,6 +32,7 @@ * This test case has been created to validate a bug that occurred when * the ReduceOperator was used without a grouping key. */ +@SuppressWarnings("serial") public class ReduceAllTest extends CompilerTestBase { @Test diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/UnionPropertyPropagationTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/UnionPropertyPropagationTest.java index 36cdb1db510e2..5fa6d343a1487 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/UnionPropertyPropagationTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/UnionPropertyPropagationTest.java @@ -44,6 +44,7 @@ import eu.stratosphere.util.Visitor; +@SuppressWarnings("serial") public class UnionPropertyPropagationTest extends CompilerTestBase { @SuppressWarnings("deprecation") diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsJavaApiCompilerTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsJavaApiCompilerTest.java new file mode 100644 index 0000000000000..3606971253703 --- /dev/null +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsJavaApiCompilerTest.java @@ -0,0 +1,300 @@ +/*********************************************************************************************************************** +* +* Copyright (C) 2013 by the Stratosphere project (http://stratosphere.eu) +* +* Licensed 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 eu.stratosphere.pact.compiler; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Iterator; + +import org.junit.Test; + +import eu.stratosphere.api.common.InvalidProgramException; +import eu.stratosphere.api.common.Plan; +import eu.stratosphere.api.common.operators.util.FieldList; +import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.DeltaIteration; +import eu.stratosphere.api.java.ExecutionEnvironment; +import eu.stratosphere.api.java.functions.GroupReduceFunction; +import eu.stratosphere.api.java.functions.JoinFunction; +import eu.stratosphere.api.java.functions.MapFunction; +import eu.stratosphere.api.java.tuple.Tuple3; +import eu.stratosphere.compiler.plan.DualInputPlanNode; +import eu.stratosphere.compiler.plan.OptimizedPlan; +import eu.stratosphere.compiler.plan.SingleInputPlanNode; +import eu.stratosphere.compiler.plantranslate.NepheleJobGraphGenerator; +import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; +import eu.stratosphere.util.Collector; + +/** +* Tests that validate optimizer choices when using operators that are requesting certain specific execution +* strategies. +*/ +@SuppressWarnings("serial") +public class WorksetIterationsJavaApiCompilerTest extends CompilerTestBase { + + private static final String JOIN_WITH_INVARIANT_NAME = "Test Join Invariant"; + private static final String JOIN_WITH_SOLUTION_SET = "Test Join SolutionSet"; + private static final String NEXT_WORKSET_REDUCER_NAME = "Test Reduce Workset"; + private static final String SOLUTION_DELTA_MAPPER_NAME = "Test Map Delta"; + + @Test + public void testJavaApiWithDeferredSoltionSetUpdateWithMapper() { + try { + Plan plan = getJavaTestPlan(false, true); + + OptimizedPlan oPlan = compileNoStats(plan); + + OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); + DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); + DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); + SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); + SingleInputPlanNode deltaMapper = resolver.getNode(SOLUTION_DELTA_MAPPER_NAME); + + // iteration preserves partitioning in reducer, so the first partitioning is out of the loop, + // the in-loop partitioning is before the final reducer + + // verify joinWithInvariant + assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy()); + assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); + assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); + + // verify joinWithSolutionSet + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getInput1().getShipStrategy()); + assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); + assertEquals(new FieldList(0, 1), joinWithSolutionSetNode.getKeysForInput1()); + + + // verify reducer + assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); + assertEquals(new FieldList(1, 2), worksetReducer.getKeys()); + + // currently, the system may partition before or after the mapper + ShipStrategyType ss1 = deltaMapper.getInput().getShipStrategy(); + ShipStrategyType ss2 = deltaMapper.getOutgoingChannels().get(0).getShipStrategy(); + + assertTrue( (ss1 == ShipStrategyType.FORWARD && ss2 == ShipStrategyType.PARTITION_HASH) || + (ss2 == ShipStrategyType.FORWARD && ss1 == ShipStrategyType.PARTITION_HASH) ); + + new NepheleJobGraphGenerator().compileJobGraph(oPlan); + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Test errored: " + e.getMessage()); + } + } + + @Test + public void testRecordApiWithDeferredSoltionSetUpdateWithNonPreservingJoin() { + try { + Plan plan = getJavaTestPlan(false, false); + + OptimizedPlan oPlan = compileNoStats(plan); + + OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); + DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); + DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); + SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); + + // iteration preserves partitioning in reducer, so the first partitioning is out of the loop, + // the in-loop partitioning is before the final reducer + + // verify joinWithInvariant + assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy()); + assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); + assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); + + // verify joinWithSolutionSet + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getInput1().getShipStrategy()); + assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); + assertEquals(new FieldList(0, 1), joinWithSolutionSetNode.getKeysForInput1()); + + // verify reducer + assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); + assertEquals(new FieldList(1, 2), worksetReducer.getKeys()); + + // verify solution delta + assertEquals(2, joinWithSolutionSetNode.getOutgoingChannels().size()); + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy()); + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getOutgoingChannels().get(1).getShipStrategy()); + + new NepheleJobGraphGenerator().compileJobGraph(oPlan); + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Test errored: " + e.getMessage()); + } + } + + @Test + public void testRecordApiWithDirectSoltionSetUpdate() { + try { + Plan plan = getJavaTestPlan(true, false); + + OptimizedPlan oPlan = compileNoStats(plan); + + + OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); + DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); + DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); + SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); + + // iteration preserves partitioning in reducer, so the first partitioning is out of the loop, + // the in-loop partitioning is before the final reducer + + // verify joinWithInvariant + assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy()); + assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); + assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); + + // verify joinWithSolutionSet + assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getInput1().getShipStrategy()); + assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); + assertEquals(new FieldList(0, 1), joinWithSolutionSetNode.getKeysForInput1()); + + // verify reducer + assertEquals(ShipStrategyType.FORWARD, worksetReducer.getInput().getShipStrategy()); + assertEquals(new FieldList(1, 2), worksetReducer.getKeys()); + + + // verify solution delta + assertEquals(1, joinWithSolutionSetNode.getOutgoingChannels().size()); + assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy()); + + new NepheleJobGraphGenerator().compileJobGraph(oPlan); + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Test errored: " + e.getMessage()); + } + } + + + @Test + public void testRejectPlanIfSolutionSetKeysAndJoinKeysDontMatch() { + try { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setDegreeOfParallelism(DEFAULT_PARALLELISM); + + @SuppressWarnings("unchecked") + DataSet> solutionSetInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Solution Set"); + @SuppressWarnings("unchecked") + DataSet> worksetInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Workset"); + @SuppressWarnings("unchecked") + DataSet> invariantInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Invariant Input"); + + DeltaIteration, Tuple3> iter = solutionSetInput.iterateDelta(worksetInput, 100, 1, 2); + + + DataSet> result = + + iter.getWorkset().join(invariantInput) + .where(1, 2) + .equalTo(1, 2) + .with(new JoinFunction, Tuple3, Tuple3>() { + public Tuple3 join(Tuple3 first, Tuple3 second) { + return first; + } + }); + + try { + result.join(iter.getSolutionSet()) + .where(1, 0) + .equalTo(0, 2) + .with(new JoinFunction, Tuple3, Tuple3>() { + public Tuple3 join(Tuple3 first, Tuple3 second) { + return second; + } + }); + fail("The join should be rejected with key type mismatches."); + } + catch (InvalidProgramException e) { + // expected! + } + + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Test errored: " + e.getMessage()); + } + } + + private Plan getJavaTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setDegreeOfParallelism(DEFAULT_PARALLELISM); + + @SuppressWarnings("unchecked") + DataSet> solutionSetInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Solution Set"); + @SuppressWarnings("unchecked") + DataSet> worksetInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Workset"); + @SuppressWarnings("unchecked") + DataSet> invariantInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Invariant Input"); + + DeltaIteration, Tuple3> iter = solutionSetInput.iterateDelta(worksetInput, 100, 1, 2); + + + DataSet> joinedWithSolutionSet = + + iter.getWorkset().join(invariantInput) + .where(1, 2) + .equalTo(1, 2) + .with(new JoinFunction, Tuple3, Tuple3>() { + public Tuple3 join(Tuple3 first, Tuple3 second) { + return first; + } + }) + .name(JOIN_WITH_INVARIANT_NAME) + + .join(iter.getSolutionSet()) + .where(1, 0) + .equalTo(1, 2) + .with(new JoinFunction, Tuple3, Tuple3>() { + public Tuple3 join(Tuple3 first, Tuple3 second) { + return second; + } + }) + .name(JOIN_WITH_SOLUTION_SET) + .withConstantSetSecond(joinPreservesSolutionSet ? new String[] {"0->0", "1->1", "2->2" } : null); + + DataSet> nextWorkset = joinedWithSolutionSet.groupBy(1, 2) + .reduceGroup(new GroupReduceFunction, Tuple3>() { + public void reduce(Iterator> values, Collector> out) {} + }) + .name(NEXT_WORKSET_REDUCER_NAME) + .withConstantSet("1->1","2->2","0->0"); + + + DataSet> nextSolutionSet = mapBeforeSolutionDelta ? + joinedWithSolutionSet.map(new MapFunction,Tuple3>() { public Tuple3 map(Tuple3 value) { return value; } }) + .name(SOLUTION_DELTA_MAPPER_NAME).withConstantSet("0->0","1->1","2->2") : + joinedWithSolutionSet; + + iter.closeWith(nextSolutionSet, nextWorkset) + .print(); + + return env.createProgramPlan(); + } +} + diff --git a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsCompilerTest.java b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsRecordApiCompilerTest.java similarity index 94% rename from stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsCompilerTest.java rename to stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsRecordApiCompilerTest.java index 89245b6b479b2..010439da7900e 100644 --- a/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsCompilerTest.java +++ b/stratosphere-compiler/src/test/java/eu/stratosphere/pact/compiler/WorksetIterationsRecordApiCompilerTest.java @@ -48,7 +48,9 @@ * Tests that validate optimizer choices when using operators that are requesting certain specific execution * strategies. */ -public class WorksetIterationsCompilerTest extends CompilerTestBase { +public class WorksetIterationsRecordApiCompilerTest extends CompilerTestBase { + + private static final long serialVersionUID = 1L; private static final String ITERATION_NAME = "Test Workset Iteration"; private static final String JOIN_WITH_INVARIANT_NAME = "Test Join Invariant"; @@ -59,8 +61,8 @@ public class WorksetIterationsCompilerTest extends CompilerTestBase { private final FieldList list0 = new FieldList(0); @Test - public void testWithDeferredSoltionSetUpdateWithMapper() { - Plan plan = getTestPlan(false, true); + public void testRecordApiWithDeferredSoltionSetUpdateWithMapper() { + Plan plan = getRecordTestPlan(false, true); OptimizedPlan oPlan; try { @@ -105,8 +107,8 @@ public void testWithDeferredSoltionSetUpdateWithMapper() { } @Test - public void testWithDeferredSoltionSetUpdateWithNonPreservingJoin() { - Plan plan = getTestPlan(false, false); + public void testRecordApiWithDeferredSoltionSetUpdateWithNonPreservingJoin() { + Plan plan = getRecordTestPlan(false, false); OptimizedPlan oPlan; try { @@ -149,8 +151,8 @@ public void testWithDeferredSoltionSetUpdateWithNonPreservingJoin() { } @Test - public void testWithDirectSoltionSetUpdate() { - Plan plan = getTestPlan(true, false); + public void testRecordApiWithDirectSoltionSetUpdate() { + Plan plan = getRecordTestPlan(true, false); OptimizedPlan oPlan; try { @@ -191,7 +193,7 @@ public void testWithDirectSoltionSetUpdate() { new NepheleJobGraphGenerator().compileJobGraph(oPlan); } - private Plan getTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) { + private Plan getRecordTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) { FileDataSource solutionSetInput = new FileDataSource(new DummyInputFormat(), IN_FILE, "Solution Set"); FileDataSource worksetInput = new FileDataSource(new DummyInputFormat(), IN_FILE, "Workset"); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/FieldList.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/FieldList.java index 39951fe80af30..96b0f347af614 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/FieldList.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/operators/util/FieldList.java @@ -32,7 +32,7 @@ public FieldList(int columnIndex) { add(columnIndex); } - public FieldList(int[] columnIndexes) { + public FieldList(int... columnIndexes) { this(); addAll(columnIndexes); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DeltaIteration.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DeltaIteration.java index 2107774f542b7..7d52a65c35adb 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DeltaIteration.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DeltaIteration.java @@ -14,6 +14,9 @@ **********************************************************************************************************************/ package eu.stratosphere.api.java; +import java.util.Arrays; + +import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.java.operators.Keys; import eu.stratosphere.api.java.typeutils.TypeInformation; @@ -42,7 +45,7 @@ public class DeltaIteration { DeltaIteration(ExecutionEnvironment context, TypeInformation type, DataSet solutionSet, DataSet workset, Keys keys, int maxIterations) { initialSolutionSet = solutionSet; initialWorkset = workset; - solutionSetPlaceholder = new SolutionSetPlaceHolder(context, solutionSet.getType()); + solutionSetPlaceholder = new SolutionSetPlaceHolder(context, solutionSet.getType(), this); worksetPlaceholder = new WorksetPlaceHolder(context, workset.getType()); this.keys = keys; this.maxIterations = maxIterations; @@ -131,8 +134,19 @@ public WorksetPlaceHolder getWorkset() { * @param The type of the elements in the solution set. */ public static class SolutionSetPlaceHolder extends DataSet{ - private SolutionSetPlaceHolder(ExecutionEnvironment context, TypeInformation type) { + + private final DeltaIteration deltaIteration; + + private SolutionSetPlaceHolder(ExecutionEnvironment context, TypeInformation type, DeltaIteration deltaIteration) { super(context, type); + this.deltaIteration = deltaIteration; + } + + public void checkJoinKeyFields(int[] keyFields) { + int[] ssKeys = deltaIteration.keys.computeLogicalKeyPositions(); + if (!Arrays.equals(ssKeys, keyFields)) { + throw new InvalidProgramException("The solution set must be joind with using the keys with which elements are identified."); + } } } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java index 9a4ac248d94b3..6a0701069e609 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java @@ -20,8 +20,10 @@ import eu.stratosphere.api.common.InvalidProgramException; import eu.stratosphere.api.common.operators.Operator; import eu.stratosphere.api.java.DataSet; +import eu.stratosphere.api.java.DeltaIteration.SolutionSetPlaceHolder; import eu.stratosphere.api.java.functions.JoinFunction; import eu.stratosphere.api.java.functions.KeySelector; +import eu.stratosphere.api.java.operators.Keys.FieldPositionKeys; import eu.stratosphere.api.java.operators.translation.KeyExtractingMapper; import eu.stratosphere.api.java.operators.translation.PlanJoinOperator; import eu.stratosphere.api.java.operators.translation.PlanMapOperator; @@ -688,6 +690,26 @@ protected DefaultJoin createJoinOperator(Keys keys2) { throw new InvalidProgramException("The pair of join keys are not compatible with each other."); } + + // sanity check solution set key mismatches + if (input1 instanceof SolutionSetPlaceHolder) { + if (keys1 instanceof FieldPositionKeys) { + int[] positions = ((FieldPositionKeys) keys1).computeLogicalKeyPositions(); + ((SolutionSetPlaceHolder) input1).checkJoinKeyFields(positions); + } else { + throw new InvalidProgramException("Currently, the solution set may only be joined with using tuple field positions."); + } + } + if (input2 instanceof SolutionSetPlaceHolder) { + if (keys2 instanceof FieldPositionKeys) { + int[] positions = ((FieldPositionKeys) keys2).computeLogicalKeyPositions(); + ((SolutionSetPlaceHolder) input2).checkJoinKeyFields(positions); + } else { + throw new InvalidProgramException("Currently, the solution set may only be joined with using tuple field positions."); + } + } + + return new DefaultJoin(input1, input2, keys1, keys2, joinHint); } } From f055024c0bc669b0b2b827b52e7a8262d6ea32c5 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 19 May 2014 16:39:56 +0200 Subject: [PATCH 049/182] Improved comments in Java example jobs --- .../example/java/clustering/KMeans.java | 10 ++++--- .../java/clustering/util/KMeansData.java | 5 ++++ .../java/graph/ConnectedComponents.java | 6 ++++- .../java/graph/EnumTrianglesBasic.java | 3 +++ .../example/java/graph/EnumTrianglesOpt.java | 3 +++ .../example/java/graph/PageRankBasic.java | 8 ++++-- .../graph/util/ConnectedComponentsData.java | 5 ++++ .../java/graph/util/EnumTrianglesData.java | 5 ++++ .../example/java/graph/util/PageRankData.java | 5 ++++ .../example/java/relational/TPCHQuery10.java | 3 +++ .../example/java/relational/TPCHQuery3.java | 5 +++- .../java/relational/WebLogAnalysis.java | 27 ++++++++++--------- .../java/relational/util/WebLogData.java | 5 ++++ .../example/java/wordcount/WordCount.java | 4 +++ .../java/wordcount/util/WordCountData.java | 5 ++++ 15 files changed, 80 insertions(+), 19 deletions(-) diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java index bbf5bf2f2d737..fc64b6500a732 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java @@ -57,6 +57,10 @@ * * *

    + * Usage: KMeans <points path> <centers path> <result path> <num iterations>
    + * If no parameters are provided, the program is run with default data from {@link KMeansData} and 10 iterations. + * + *

    * This example shows how to use: *

      *
    • Bulk iterations @@ -103,7 +107,7 @@ public static void main(String[] args) throws Exception { // emit result if(fileOutput) { - clusteredPoints.writeAsCsv(outputPath, "\n", ","); + clusteredPoints.writeAsCsv(outputPath, "\n", " "); } else { clusteredPoints.print(); } @@ -153,7 +157,7 @@ public void clear() { @Override public String toString() { - return x + "," + y; + return x + " " + y; } } @@ -178,7 +182,7 @@ public Centroid(int id, Point p) { @Override public String toString() { - return id + "," + super.toString(); + return id + " " + super.toString(); } } diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java index 5b37002c93db4..66e47adeb63a2 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/util/KMeansData.java @@ -18,6 +18,11 @@ import eu.stratosphere.example.java.clustering.KMeans.Centroid; import eu.stratosphere.example.java.clustering.KMeans.Point; +/** + * Provides the default data sets used for the K-Means example program. + * The default data sets are used, if no parameters are given to the program. + * + */ public class KMeansData { public static DataSet getDefaultCentroidDataSet(ExecutionEnvironment env) { diff --git a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java index e3b2c77c61798..3eb883a8fd4f6 100644 --- a/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java +++ b/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java @@ -55,6 +55,10 @@ *
    * *

    + * Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>
    + * If no parameters are provided, the program is run with default data from {@link ConnectedComponentsData} and 10 iterations. + * + *

    * This example shows how to use: *