You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mrql.apache.org by fe...@apache.org on 2014/03/13 15:24:26 UTC
[01/26] MRQL-32: Refactoring directory structure for Eclipse
Repository: incubator-mrql
Updated Branches:
refs/heads/master 44c44bdc1 -> 1adaa71cb
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/SparkEvaluator.gen
----------------------------------------------------------------------
diff --git a/src/main/java/spark/SparkEvaluator.gen b/src/main/java/spark/SparkEvaluator.gen
deleted file mode 100644
index 611eca2..0000000
--- a/src/main/java/spark/SparkEvaluator.gen
+++ /dev/null
@@ -1,762 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java_cup.runtime.Scanner;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Hashtable;
-import java.util.Iterator;
-import java.io.*;
-import java.util.Enumeration;
-import org.apache.log4j.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import scala.Tuple2;
-import org.apache.spark.TaskContext;
-import org.apache.spark.Partition;
-import org.apache.spark.Accumulator;
-import org.apache.spark.broadcast.Broadcast;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function2;
-import org.apache.spark.api.java.function.PairFunction;
-import org.apache.spark.api.java.function.FlatMapFunction;
-import org.apache.spark.api.java.function.PairFlatMapFunction;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.VoidFunction;
-
-
-/** Evaluates physical plans in Apache Spark mode */
-final public class SparkEvaluator extends Evaluator implements Serializable {
- public static JavaSparkContext spark_context;
- final static String spark_jar = SparkEvaluator.class.getProtectionDomain().getCodeSource().getLocation().toString();
- final static String core_jar = Interpreter.class.getProtectionDomain().getCodeSource().getLocation().toString();
- final static String gen_jar = Tree.class.getProtectionDomain().getCodeSource().getLocation().toString();
- final static String scanner_jar = Scanner.class.getProtectionDomain().getCodeSource().getLocation().toString();
- static String data_source_directory;
- final static Bag empty_bag = new Bag();
-
- /** initialize the Spark evaluator */
- final public void init ( Configuration conf ) {
- Config.spark_mode = true;
- try {
- if (Config.hadoop_mode && Config.local_mode)
- spark_context = new JavaSparkContext("local["+Config.nodes+"]",
- "Apache MRQL Local Spark Evaluator");
- else if (Config.hadoop_mode) {
- HashMap<String,String> env = new HashMap<String,String>();
- data_source_directory = System.getenv("FS_DEFAULT_NAME")+"/"+Plan.new_path(conf);
- env.put("mrql.data.source.directory",data_source_directory);
- spark_context = new JavaSparkContext(System.getenv("SPARK_MASTER"),
- "Apache MRQL Spark Evaluator",
- System.getenv("SPARK_HOME"),
- new String[]{spark_jar,core_jar,gen_jar,scanner_jar},
- env);
- Plan.conf = spark_context.hadoopConfiguration();
- FileSystem.setDefaultUri(Plan.conf,System.getenv("FS_DEFAULT_NAME"));
- };
- if (!Config.info) {
- for ( Enumeration en = LogManager.getCurrentLoggers(); en.hasMoreElements(); )
- ((Logger)en.nextElement()).setLevel(Level.WARN);
- LogManager.getRootLogger().setLevel(Level.WARN);
- }
- } catch (IOException ex) {
- throw new Error(ex);
- }
- }
-
- /** shutdown the Spark evaluator */
- final public void shutdown ( Configuration conf ) {
- spark_context.stop();
- spark_context = null;
- System.clearProperty("spark.driver.port");
- }
-
- final public void initialize_query () {
- Plan.distribute_compiled_arguments(Plan.conf);
- if (Config.compile_functional_arguments)
- spark_context.addJar(Plan.conf.get("mrql.jar.path"));
- }
-
- final public Configuration new_configuration () {
- return new Configuration();
- }
-
- /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
- final public Class<? extends MRQLFileInputFormat> parsedInputFormat () {
- return SparkParsedInputFormat.class;
- }
-
- /** return the FileInputFormat for binary files */
- final public Class<? extends MRQLFileInputFormat> binaryInputFormat () {
- return SparkBinaryInputFormat.class;
- }
-
- /** return the FileInputFormat for data generator files */
- final public Class<? extends MRQLFileInputFormat> generatorInputFormat () {
- return SparkGeneratorInputFormat.class;
- }
-
- /** used by the master to send parsing details (eg, record types) to workers */
- public static void dump_source_dir () throws IOException {
- if (Config.local_mode)
- return;
- DataSource.dataSourceDirectory.distribute(Plan.conf);
- Path path = new Path(data_source_directory);
- FileSystem fs = path.getFileSystem(Plan.conf);
- PrintStream ps = new PrintStream(fs.create(path,true));
- ps.println(Plan.conf.get("mrql.data.source.directory"));
- ps.close();
- }
-
- /** executed by a worker when reading parsed input (see SparkParsedInputFormat) */
- public static void load_source_dir () throws IOException {
- if (Config.local_mode)
- return;
- if (Plan.conf == null) {
- if (evaluator == null)
- evaluator = new SparkEvaluator();
- Plan.conf = evaluator.new_configuration();
- };
- // the name of the file that contains the source directory details is passed
- // to workers through the HashMap environment in the JavaSparkContext
- Path path = new Path(System.getenv("mrql.data.source.directory"));
- FileSystem fs = path.getFileSystem(Plan.conf);
- BufferedReader ftp = new BufferedReader(new InputStreamReader(fs.open(path)));
- Plan.conf.set("mrql.data.source.directory",ftp.readLine());
- DataSource.dataSourceDirectory.read(Plan.conf);
- ftp.close();
- }
-
- /** dump MRQL data into a sequence file */
- @Override
- final public void dump ( String file, Tree type, MRData data ) throws Exception {
- if (data instanceof MR_dataset && ((MR_dataset)data).dataset().source.get(0) instanceof RDDDataSource) {
- Path path = new Path(file);
- FileSystem fs = path.getFileSystem(Plan.conf);
- PrintStream ftp = new PrintStream(fs.create(path.suffix(".type")));
- fs.delete(path,true);
- ftp.print("2@"+type.toString()+"\n");
- ftp.close();
- // in Spark mode, a dataset has always one RDD datasource
- JavaRDD<MRData> rdd = ((RDDDataSource)((MR_dataset)data).dataset().source.get(0)).rdd.cache();
- rdd.map(new PairFunction<MRData,MRContainer,MRContainer>() {
- final MRContainer zero = new MRContainer(new MR_int(0));
- public Tuple2<MRContainer,MRContainer> call ( MRData value ) {
- return new Tuple2<MRContainer,MRContainer>(zero,new MRContainer(value));
- }
- })
- .saveAsHadoopFile(file,MRContainer.class,MRContainer.class,SequenceFileOutputFormat.class);
- } else super.dump(file,type,data);
- }
-
- /** dump MRQL data into text a CVS file */
- @Override
- final public void dump_text ( String file, Tree type, MRData data ) throws Exception {
- if (data instanceof MR_dataset && ((MR_dataset)data).dataset().source.get(0) instanceof RDDDataSource) {
- // in Spark mode, a dataset has always one RDD datasource
- JavaRDD<MRData> rdd = ((RDDDataSource)((MR_dataset)data).dataset().source.get(0)).rdd.cache();
- int ps = Config.max_bag_size_print;
- Config.max_bag_size_print = -1;
- match type {
- case `T(tuple(...tps)):
- final Trees ts = tps;
- rdd.map(new Function<MRData,String>() {
- public String call ( MRData value ) {
- Tuple t = (Tuple)value;
- String s = Printer.print(t.get((short)0),ts.nth(0));
- for ( short i = 1; i < t.size(); i++ )
- s += ","+Printer.print(t.get(i),ts.nth(i));
- return s;
- }
- }).saveAsTextFile(file);
- case _:
- rdd.saveAsTextFile(file);
- };
- Config.max_bag_size_print = ps;
- } else super.dump_text(file,type,data);
- }
-
- private static Function2<MRData,MRData,MRData> accumulator ( final Tree acc_fnc, final Environment env ) {
- final org.apache.mrql.Function f = evalF(acc_fnc,env);
- return new Function2<MRData,MRData,MRData>() {
- public MRData call ( MRData x, MRData y ) {
- return f.eval(new Tuple(x,y));
- }
- };
- }
-
- /** The Aggregate physical operator
- * @param acc_fnc the accumulator function from (T,T) to T
- * @param zero the zero element of type T
- * @param plan the plan that constructs the dataset that contains the bag of values {T}
- * @param env contains bindings fro variables to values (MRData)
- * @return the aggregation result of type T
- */
- final public MRData aggregate ( Tree acc_fnc,
- Tree zero,
- Tree plan,
- Environment env ) throws Exception {
- Function2<MRData,MRData,MRData> f2 = accumulator(acc_fnc,env);
- MRData z = evalE(zero,env);
- match plan {
- case AggregateMap(`m,`acc,_,`s):
- return evalD(#<cMap(`m,`s)>,env)
- .aggregate(z,accumulator(acc,env),f2);
- case MapAggregateReduce(`m,`r,`acc,_,`s,`o):
- if (acc.equals(#<null>))
- fail;
- return evalD(#<MapReduce(`m,`r,`s,`o)>,env)
- .aggregate(z,accumulator(acc,env),f2);
- case CrossAggregateProduct(`mx,`my,`r,`acc,_,`x,`y):
- if (acc.equals(#<null>))
- fail;
- return evalD(#<CrossProduct(`mx,`my,`r,`x,`y)>,env)
- .aggregate(z,accumulator(acc,env),f2);
- case MapAggregateReduce2(`mx,`my,`r,`acc,_,`x,`y,`o):
- if (acc.equals(#<null>))
- fail;
- return evalD(#<MapReduce2(`mx,`my,`r,`x,`y,`o)>,env)
- .aggregate(z,accumulator(acc,env),f2);
- case MapAggregateJoin(`mx,`my,`r,`acc,_,`x,`y):
- if (acc.equals(#<null>))
- fail;
- return evalD(#<MapJoin(`mx,`my,`r,`x,`y)>,env)
- .aggregate(z,accumulator(acc,env),f2);
- };
- throw new Error("Unrecognized aggregation: "+plan);
- }
-
- /** Evaluate a loop a fixed # of times */
- final public Tuple loop ( Tree e, Environment env ) throws Exception {
- match e {
- case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`num):
- int limit = ((MR_int)evalE(num,env)).get();
- MR_rdd[] s = new MR_rdd[vs.length()];
- for ( int i = 0; i < vs.length(); i++ )
- s[i] = new MR_rdd(eval(ss.nth(i),env));
- for ( int n = 0; n < limit; n++ ) {
- Environment nenv = env;
- for ( int i = 0; i < vs.length(); i ++ )
- nenv = new Environment(vs.nth(i).toString(),s[i],nenv);
- for ( int i = 0; i < vs.length(); i ++ )
- s[i] = new MR_rdd(eval(bs.nth(i),nenv));
- };
- return new Tuple(s);
- };
- throw new Error("Wrong Loop format");
- }
-
- private static Bag bag ( final List<MRData> s ) {
- final Iterator<MRData> i = s.iterator();
- return new Bag(new BagIterator() {
- public MRData next () {
- return i.next();
- }
- public boolean hasNext () {
- return i.hasNext();
- }
- });
- }
-
- // changed since Spark 0.8.1
- final static TaskContext context = new TaskContext(0,0,(long)0,Config.local_mode,false,null);
-
- /** Convert a Spark RDD into a lazy bag
- * @param rdd the Spark RDD
- * @return a lazy bag that contains all RDD elements
- */
- public static Bag bag ( final JavaRDD<MRData> rdd ) {
- final JavaRDD<MRData> rd = rdd.cache();
- rd.count(); // force the evaluation of all pending Spark operations
- final List<Partition> ps = rd.splits();
- return new Bag(new BagIterator() {
- Iterator<MRData> i = null;
- int c = 0;
- public MRData next () {
- return i.next();
- }
- public boolean hasNext () {
- do {
- if (i != null && i.hasNext())
- return true;
- if (c >= ps.size())
- return false;
- i = rdd.iterator(ps.get(c++),context);
- } while (true);
- }
- });
- }
-
- private final static Function<MRData,MRData> get_first
- = new Function<MRData,MRData>() {
- public MRData call ( MRData value ) {
- return ((Tuple)value).first();
- };
- };
-
- /** Evaluate an MRQL physical plan using Spark
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @param counter the name of the counter used in loops
- * @return a DataSet
- */
- final public DataSet eval ( final Tree e,
- final Environment env,
- final String counter ) {
- JavaRDD<MRData> rd = eval(e,env);
- int count = 0;
- if (!counter.equals("-")) {
- final Accumulator<Integer> c = spark_context.intAccumulator(0);
- rd = rd.cache();
- rd.foreach(new VoidFunction<MRData>() {
- public void call ( MRData value ) {
- if (((MR_bool)((Tuple)value).second()).get())
- c.add(1); // count the true results (the results that need another loop step)
- }
- });
- count = c.value();
- rd = rd.map(get_first);
- };
- return new DataSet(new RDDDataSource(rd),count,0);
- }
-
- /** Evaluate an MRQL physical plan using Spark and print tracing info
- * @param e the physical plan
- * @param env contains bindings from variables to values (MRData)
- * @return a Spark RDD
- */
- final public JavaRDD<MRData> eval ( final Tree e, final Environment env ) {
- if (Config.trace_execution) {
- tab_count += 3;
- System.out.println(tabs(tab_count)+print_query(e));
- };
- final JavaRDD<MRData> res = evalD(e,env);
- if (Config.trace_execution)
- try {
- System.out.println(tabs(tab_count)+"-> "+res.take(Config.max_bag_size_print));
- tab_count -= 3;
- } catch (Exception ex) {
- throw new Error("Cannot collect the operator output: "+e);
- };
- return res;
- }
-
- /* convert an MRQL lambda to a Spark Function */
- private static FlatMapFunction<MRData,MRData> cmap_fnc ( Tree fnc, Environment env ) {
- final org.apache.mrql.Function f = evalF(fnc,env);
- return new FlatMapFunction<MRData,MRData>() {
- public Iterable<MRData> call ( MRData value ) {
- return (Bag)f.eval(value);
- }
- };
- }
-
- /* group-by s and then reduce by fnc; if o is true, sort after group-by */
- private static JavaRDD<MRData> groupBy ( JavaRDD<MRData> s, Tree fnc, Environment env, Tree o ) {
- match o {
- case true: // the result must be sorted
- return s.groupBy(get_first)
- .sortByKey()
- .map(new Function<Tuple2<MRData,List<MRData>>,MRData>() {
- public MRData call ( Tuple2<MRData,List<MRData>> value ) {
- return new Tuple(value._1,bag(value._2));
- }
- })
- .flatMap(cmap_fnc(fnc,env)).map(new Function<MRData,MRData>() {
- public MRData call ( MRData value ) {
- return ((Tuple)value).second();
- }
- });
- };
- return s.groupBy(get_first)
- .map(new Function<Tuple2<MRData,List<MRData>>,MRData> () {
- public MRData call ( Tuple2<MRData,List<MRData>> value ) {
- final Iterator<MRData> i = value._2.iterator();
- return new Tuple(value._1,new Bag(new BagIterator() {
- public MRData next () {
- return ((Tuple)i.next()).second();
- }
- public boolean hasNext () {
- return i.hasNext();
- }
- }));
- }
- })
- .flatMap(cmap_fnc(fnc,env));
- }
-
- private static JavaRDD<MRData> containerData ( JavaPairRDD<MRContainer,MRContainer> rd ) {
- final Environment env = Interpreter.global_env;
- return rd.map(new Function<Tuple2<MRContainer,MRContainer>,MRData>() {
- boolean first = true;
- public MRData call ( Tuple2<MRContainer,MRContainer> value ) {
- if (first) {
- // need to pass the global bindings (the in-memory repeat vars) to workers
- Interpreter.set_global_bindings(env);
- first = false;
- };
- return value._2.data();
- }
- });
- }
-
- private static Iterable<Tuple2<MRData,MRData>> joinIterator ( final Iterator<MRData> i ) {
- return new Iterable<Tuple2<MRData,MRData>>() {
- public Iterator<Tuple2<MRData,MRData>> iterator() {
- return new Iterator<Tuple2<MRData,MRData>> () {
- public Tuple2<MRData,MRData> next () {
- Tuple data = (Tuple)i.next();
- return new Tuple2<MRData,MRData>(data.first(),data.second());
- }
- public boolean hasNext () {
- return i.hasNext();
- }
- public void remove () {}
- };
- }
- };
- }
-
- private static FlatMapFunction<Iterator<MRData>,MRData> combiner_fnc ( final org.apache.mrql.Function f ) {
- return new FlatMapFunction<Iterator<MRData>,MRData>() {
- public Iterable<MRData> call ( final Iterator<MRData> i ) {
- return MapReduceAlgebra.cmap(new org.apache.mrql.Function() {
- public MRData eval ( MRData x ) {
- final MRData key = ((Tuple)x).first();
- final Iterator<MRData> it = ((Bag)f.eval(x)).iterator();
- return new Bag(new BagIterator() {
- public MRData next () {
- return new Tuple(key,it.next());
- }
- public boolean hasNext () {
- return it.hasNext();
- }
- });
- }},
- MapReduceAlgebra.groupBy(new Bag(new BagIterator() {
- public MRData next () {
- return i.next();
- }
- public boolean hasNext () {
- return i.hasNext();
- }
- })));
- }
- };
- }
-
- private static Hashtable<MRData,Bag> make_built_table ( List<Tuple2<MRData,MRData>> values ) {
- Hashtable<MRData,Bag> built_table = new Hashtable<MRData,Bag>(Config.map_cache_size);
- for ( Tuple2<MRData,MRData> t: values ) {
- Bag entries = built_table.get(t._1);
- built_table.put(t._1,
- (entries == null)
- ? (new Bag(t._2))
- : entries.add_element(t._2));
- };
- return built_table;
- }
-
- /** Evaluate MRQL physical operators using Spark
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a Spark RDD
- */
- final public JavaRDD<MRData> evalD ( final Tree e, final Environment env ) {
- try {
- match e {
- case MapAggregateReduce(`m,`r,null,_,`s,`o):
- return evalD(#<MapReduce(`m,`r,`s,`o)>,env);
- case CrossAggregateProduct(`mx,`my,`r,null,_,`x,`y):
- return evalD(#<CrossProduct(`mx,`my,`r,`x,`y)>,env);
- case MapAggregateReduce2(`mx,`my,`r,null,_,`x,`y,`o):
- return evalD(#<MapReduce2(`mx,`my,`r,`x,`y,`o)>,env);
- case MapAggregateJoin(`mx,`my,`r,null,_,`x,`y):
- return evalD(#<MapJoin(`mx,`my,`r,`x,`y)>,env);
- case cMap(`f,`s):
- return eval(s,env).flatMap(cmap_fnc(f,env));
- case MapReduce(`m,`r,`s,`o):
- return groupBy(eval(s,env).flatMap(cmap_fnc(m,env)),r,env,o);
- case MapCombineReduce(`m,`c,`r,`s,`o):
- return groupBy(eval(s,env).flatMap(cmap_fnc(m,env))
- .mapPartitions(combiner_fnc(evalF(c,env))),r,env,o);
- case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,`o):
- return evalD(#<MapReduce2(`mx,`my,`r,`x,`y,`o)>,env).mapPartitions(combiner_fnc(evalF(c,env)));
- case CrossProduct(`mx,`my,`r,`x,`y):
- final org.apache.mrql.Function fr = evalF(r,env);
- return eval(x,env)
- .flatMap(cmap_fnc(mx,env))
- .cartesian(eval(y,env).flatMap(cmap_fnc(my,env)))
- .flatMap(new FlatMapFunction<Tuple2<MRData,MRData>,MRData>() {
- public Iterable<MRData> call ( Tuple2<MRData,MRData> value ) {
- return (Bag)fr.eval(new Tuple(value._1,value._2));
- }
- });
- case MapReduce2(`mx,`my,`r,`x,`y,`o):
- final org.apache.mrql.Function fx = evalF(mx,env);
- final org.apache.mrql.Function fy = evalF(my,env);
- final org.apache.mrql.Function fr = evalF(r,env);
- JavaPairRDD<MRData,MRData> xs
- = eval(x,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
- public Iterable<Tuple2<MRData,MRData>> call ( MRData value ) {
- return joinIterator(((Bag)fx.eval(value)).iterator());
- }
- });
- JavaPairRDD<MRData,MRData> ys
- = eval(y,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
- public Iterable<Tuple2<MRData,MRData>> call ( MRData value ) {
- return joinIterator(((Bag)fy.eval(value)).iterator());
- }
- });
- return xs.cogroup(ys)
- .flatMap(new FlatMapFunction<Tuple2<MRData,Tuple2<List<MRData>,List<MRData>>>,MRData>() {
- public Iterable<MRData> call ( Tuple2<MRData,Tuple2<List<MRData>,List<MRData>>> value ) {
- return (Bag)fr.eval(new Tuple(bag(value._2._1),bag(value._2._2)));
- }
- });
- case GroupByJoin(`kx,`ky,`gx,`gy,`mp,`c,`r,`x,`y,`o):
- final int n = (int)Math.floor(Math.sqrt(4.0*Config.nodes));
- final int m = n;
- if (Config.trace)
- System.out.println("*** Using a groupBy join on a "+n+"*"+m+" grid of partitions");
- final org.apache.mrql.Function fkx = evalF(kx,env);
- final org.apache.mrql.Function fky = evalF(ky,env);
- final org.apache.mrql.Function fgx = evalF(gx,env);
- final org.apache.mrql.Function fgy = evalF(gy,env);
- final org.apache.mrql.Function fm = evalF(mp,env);
- final org.apache.mrql.Function fc = evalF(c,env);
- final org.apache.mrql.Function fr = evalF(r,env);
- final MRData one = new MR_byte(1);
- final MRData two = new MR_byte(2);
- final JavaPairRDD<MRData,MRData> xs
- = eval(x,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
- public Iterable<Tuple2<MRData,MRData>> call ( final MRData value ) {
- return new Iterable<Tuple2<MRData,MRData>>() {
- public Iterator<Tuple2<MRData,MRData>> iterator() {
- return new Iterator<Tuple2<MRData,MRData>>() {
- int i = 0;
- public Tuple2<MRData,MRData> next () {
- MRData key = new MR_int((fgx.eval(value).hashCode() % m)+m*i);
- i++;
- return new Tuple2<MRData,MRData>(key,new Tuple(one,value));
- }
- public boolean hasNext () {
- return i < n;
- }
- public void remove () {}
- };
- }
- };
- }
- });
- final JavaPairRDD<MRData,MRData> ys
- = eval(y,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
- public Iterable<Tuple2<MRData,MRData>> call ( final MRData value ) {
- return new Iterable<Tuple2<MRData,MRData>>() {
- public Iterator<Tuple2<MRData,MRData>> iterator() {
- return new Iterator<Tuple2<MRData,MRData>>() {
- int j = 0;
- public Tuple2<MRData,MRData> next () {
- MRData key = new MR_int((fgy.eval(value).hashCode() % n)*m+j);
- j++;
- return new Tuple2<MRData,MRData>(key,new Tuple(two,value));
- }
- public boolean hasNext () {
- return j < m;
- }
- public void remove () {}
- };
- }
- };
- }
- });
- return xs.union(ys).groupByKey(Config.nodes)
- .mapPartitions(new FlatMapFunction<Iterator<Tuple2<MRData,List<MRData>>>,MRData>() {
- public Iterable<MRData> call ( final Iterator<Tuple2<MRData,List<MRData>>> value ) {
- Bag xb = new Bag();
- Bag yb = new Bag();
- while (value.hasNext()) {
- Tuple2<MRData,List<MRData>> t = value.next();
- for ( MRData e: t._2 ) {
- Tuple p = (Tuple)e;
- if (((MR_byte)p.first()).get() == 1)
- xb.add(new Tuple(t._1,p.second()));
- else yb.add(new Tuple(t._1,p.second()));
- }
- };
- final Bag b = MapReduceAlgebra.mergeGroupByJoin(fkx,fky,fgx,fgy,fm,fc,fr,xb,yb);
- xb = null; yb = null;
- return new Iterable<MRData>() {
- public Iterator<MRData> iterator() {
- return b.iterator();
- }
- };
- }
- });
- case MapJoin(`mx,`my,`r,`x,`y):
- final org.apache.mrql.Function fx = evalF(mx,env);
- final org.apache.mrql.Function fy = evalF(my,env);
- final org.apache.mrql.Function fr = evalF(r,env);
- final Broadcast<List<Tuple2<MRData,MRData>>> ys
- = spark_context.broadcast(eval(y,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
- public Iterable<Tuple2<MRData,MRData>> call ( MRData value ) {
- return joinIterator(((Bag)fy.eval(value)).iterator());
- }
- }).collect());
- return eval(x,env).flatMap(new FlatMapFunction<MRData,MRData>() {
- final Hashtable<MRData,Bag> built_table = make_built_table(ys.value());
- public Iterable<MRData> call ( MRData value ) {
- final Iterator<MRData> i = ((Bag)fx.eval(value)).iterator();
- return new Iterable<MRData>() {
- public Iterator<MRData> iterator() {
- return new Iterator<MRData>() {
- Tuple p;
- Iterator<MRData> ix = null;
- public MRData next () {
- return ix.next();
- }
- public boolean hasNext () {
- if (ix != null && ix.hasNext())
- return true;
- while (i.hasNext()) {
- p = (Tuple)i.next();
- MRData pd = built_table.get(p.first());
- Bag bb = ((Bag)fr.eval(new Tuple(p.second(),(pd == null) ? empty_bag : pd)));
- ix = bb.iterator();
- if (ix.hasNext())
- return true;
- };
- return false;
- }
- public void remove () {}
- };
- }
- };
- }
- });
- case BinarySource(`file,_):
- String path = ((MR_string)evalE(file,env)).get();
- new BinaryDataSource(path,Plan.conf);
- return containerData(spark_context.sequenceFile(file.stringValue(),
- MRContainer.class,MRContainer.class,
- Config.nodes));
- case ParsedSource(`parser,`file,...args):
- String path = ((MR_string)evalE(file,env)).get();
- Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- new ParsedDataSource(path,p,args,Plan.conf);
- dump_source_dir();
- return containerData(spark_context.hadoopFile(path,SparkParsedInputFormat.class,
- MRContainer.class,MRContainer.class,
- Config.nodes));
- case Merge(`x,`y):
- return eval(x,env).union(eval(y,env));
- case Repeat(lambda(`v,`b),`s,`n):
- int max_num = ((MR_int)evalE(n,env)).get();
- JavaRDD<MRData> rd;
- JavaRDD<MRData> res = eval(s,env).cache();
- int i = 0;
- boolean cont = true;
- do {
- rd = eval(b,new Environment(v.toString(),new MR_rdd(res),env)).cache();
- res = rd.map(get_first).cache();
- Integer true_results
- = rd.aggregate(new Integer(0),
- new Function2<Integer,MRData,Integer>() {
- public Integer call ( Integer x, MRData y ) {
- return (((MR_bool)((Tuple)y).second()).get()) ? x+1 : x;
- }
- },
- new Function2<Integer,Integer,Integer>() {
- public Integer call ( Integer x, Integer y ) { return x+y; }
- });
- i++;
- cont = true_results > 0 && i <= max_num;
- System.err.println("*** Repeat #"+i+": "+true_results+" true results");
- } while (cont);
- return res;
- case Closure(lambda(`v,`b),`s,`m):
- int max_num = ((MR_int)evalE(m,env)).get();
- JavaRDD<MRData> res = eval(s,env).cache();
- long n = 0;
- long old = 0;
- int i = 0;
- boolean cont = true;
- do {
- res = eval(b,new Environment(v.toString(),new MR_rdd(res),env)).cache();
- old = n;
- n = res.count();
- i++;
- System.err.println("*** Repeat #"+i+": "+(old-n)+" new records");
- } while (old < n && i <= max_num);
- return res;
- case Generator(`min,`max,`size):
- DataSet ds = Plan.generator(((MR_long)evalE(min,env)).get(),
- ((MR_long)evalE(max,env)).get(),
- ((MR_long)evalE(size,env)).get());
- JavaRDD<MRData> rd = null;
- for ( DataSource d: ds.source )
- if (rd == null)
- rd = containerData(spark_context.hadoopFile(d.path,SparkGeneratorInputFormat.class,
- MRContainer.class,MRContainer.class,1));
- else rd = rd.union(containerData(spark_context.hadoopFile(d.path,SparkGeneratorInputFormat.class,
- MRContainer.class,MRContainer.class,1)));
- return rd;
- case let(`v,`u,`body):
- return eval(body,new Environment(v.toString(),evalE(u,env),env));
- case Let(`v,`u,`body):
- return eval(body,new Environment(v.toString(),new MR_rdd(eval(u,env)),env));
- case If(`c,`x,`y):
- if (((MR_bool)evalE(c,env)).get())
- return eval(x,env);
- else return eval(y,env);
- case `v:
- if (!v.is_variable())
- fail;
- MRData x = variable_lookup(v.toString(),env);
- if (x != null)
- if (x instanceof MR_rdd)
- return ((MR_rdd)x).rdd();
- x = variable_lookup(v.toString(),global_env);
- if (x != null)
- if (x instanceof MR_rdd)
- return ((MR_rdd)x).rdd();
- throw new Error("Variable "+v+" is not bound");
- };
- throw new Error("Cannot evaluate the Spark plan: "+e);
- } catch (Error msg) {
- if (!Config.trace)
- throw new Error(msg.getMessage());
- System.err.println(msg.getMessage());
- throw new Error("Evaluation error in: "+print_query(e));
- } catch (Exception ex) {
- System.err.println(ex.getMessage());
- ex.printStackTrace();
- throw new Error("Evaluation error in: "+print_query(e));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/SparkGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/spark/SparkGeneratorInputFormat.java b/src/main/java/spark/SparkGeneratorInputFormat.java
deleted file mode 100644
index 8aaca3c..0000000
--- a/src/main/java/spark/SparkGeneratorInputFormat.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.*;
-
-
-/** the FileInputFormat for data generators: it creates HDFS files, where each file contains
- * an (offset,size) pair that generates the range of values [offset,offset+size] */
-final public class SparkGeneratorInputFormat extends SparkMRQLFileInputFormat {
- public static class GeneratorRecordReader implements RecordReader<MRContainer,MRContainer> {
- final long offset;
- final long size;
- long index;
- SequenceFile.Reader reader;
-
- public GeneratorRecordReader ( FileSplit split,
- Configuration conf ) throws IOException {
- Path path = split.getPath();
- FileSystem fs = path.getFileSystem(conf);
- reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
- MRContainer key = new MRContainer();
- MRContainer value = new MRContainer();
- reader.next(key,value);
- offset = ((MR_long)((Tuple)(value.data())).first()).get();
- size = ((MR_long)((Tuple)(value.data())).second()).get();
- index = -1;
- }
-
- public MRContainer createKey () {
- return new MRContainer(null);
- }
-
- public MRContainer createValue () {
- return new MRContainer(null);
- }
-
- public boolean next ( MRContainer key, MRContainer value ) throws IOException {
- index++;
- value.set(new MR_long(offset+index));
- key.set(new MR_long(index));
- return index < size;
- }
-
- public long getPos () throws IOException { return index; }
-
- public void close () throws IOException { reader.close(); }
-
- public float getProgress () throws IOException {
- return index / (float)size;
- }
- }
-
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split, JobConf job, Reporter reporter ) throws IOException {
- return (RecordReader<MRContainer,MRContainer>)
- new GeneratorRecordReader((FileSplit)split,job);
- }
-
- /** Insert all results from the generators stored in path into a Bag.
- * The Bag is lazily constructed.
- * @param path the path directory that contains the generator data (offset,size)
- * @return a Bag that contains all data
- */
- public Bag materialize ( final Path path ) throws IOException {
- Configuration conf = Plan.conf;
- FileSystem fs = path.getFileSystem(conf);
- final SequenceFile.Reader reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
- final MRContainer key = new MRContainer();
- final MRContainer value = new MRContainer();
- return new Bag(new BagIterator () {
- long offset = 0;
- long size = 0;
- long i = 0;
- public boolean hasNext () {
- if (++i >= offset+size)
- try {
- if (!reader.next(key,value))
- return false;
- offset = ((MR_long)((Tuple)(value.data())).first()).get();
- size = ((MR_long)((Tuple)(value.data())).second()).get();
- i = offset;
- } catch (IOException e) {
- throw new Error("Cannot collect values from a generator");
- };
- return true;
- }
- public MRData next () {
- return new MR_long(i);
- }
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/SparkMRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/spark/SparkMRQLFileInputFormat.java b/src/main/java/spark/SparkMRQLFileInputFormat.java
deleted file mode 100644
index af5ebfa..0000000
--- a/src/main/java/spark/SparkMRQLFileInputFormat.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
-
-
-/** A superclass for all MRQL FileInputFormats */
-abstract public class SparkMRQLFileInputFormat extends FileInputFormat<MRContainer,MRContainer> implements MRQLFileInputFormat {
- public SparkMRQLFileInputFormat () {}
-
- /** record reader for spark */
- abstract public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split, JobConf job, Reporter reporter ) throws IOException;
-
- /** materialize the input file into a memory Bag */
- public Bag materialize ( final Path file ) throws IOException {
- final JobConf job = new JobConf(Plan.conf,MRQLFileInputFormat.class);
- setInputPaths(job,file);
- final InputSplit[] splits = getSplits(job,1);
- final Reporter reporter = null;
- final RecordReader<MRContainer,MRContainer> rd = getRecordReader(splits[0],job,reporter);
- return new Bag(new BagIterator () {
- RecordReader<MRContainer,MRContainer> reader = rd;
- MRContainer key = reader.createKey();
- MRContainer value = reader.createKey();
- int i = 0;
- public boolean hasNext () {
- try {
- if (reader.next(key,value))
- return true;
- do {
- if (++i >= splits.length)
- return false;
- reader.close();
- reader = getRecordReader(splits[i],job,reporter);
- } while (!reader.next(key,value));
- return true;
- } catch (IOException e) {
- throw new Error("Cannot collect values from an intermediate result");
- }
- }
- public MRData next () {
- return value.data();
- }
- });
- }
-
- /** materialize the entire dataset into a Bag
- * @param x the DataSet in HDFS to collect values from
- * @param strip is not used in MapReduce mode
- * @return the Bag that contains the collected values
- */
- public final Bag collect ( final DataSet x, boolean strip ) throws Exception {
- Bag res = new Bag();
- for ( DataSource s: x.source )
- if (s instanceof RDDDataSource)
- res = res.union(SparkEvaluator.bag(((RDDDataSource)s).rdd));
- else if (s.to_be_merged)
- res = res.union(Plan.merge(s));
- else res = res.union(s.inputFormat.newInstance().materialize(new Path(s.path)));
- return res;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/SparkParsedInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/spark/SparkParsedInputFormat.java b/src/main/java/spark/SparkParsedInputFormat.java
deleted file mode 100644
index 995f69c..0000000
--- a/src/main/java/spark/SparkParsedInputFormat.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.Trees;
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** A FileInputFormat for text files (CVS, XML, JSON, ...) */
-final public class SparkParsedInputFormat extends SparkMRQLFileInputFormat {
- public static class ParsedRecordReader implements RecordReader<MRContainer,MRContainer> {
- final FSDataInputStream fsin;
- final long start;
- final long end;
- Iterator<MRData> result;
- Parser parser;
-
- public ParsedRecordReader ( FileSplit split,
- Configuration conf,
- Class<? extends Parser> parser_class,
- Trees args ) throws IOException {
- start = split.getStart();
- end = start + split.getLength();
- Path file = split.getPath();
- FileSystem fs = file.getFileSystem(conf);
- fsin = fs.open(split.getPath());
- try {
- parser = parser_class.newInstance();
- } catch (Exception ex) {
- throw new Error("Unrecognized parser:"+parser_class);
- };
- parser.initialize(args);
- parser.open(fsin,start,end);
- result = null;
- }
-
- public MRContainer createKey () {
- return new MRContainer();
- }
-
- public MRContainer createValue () {
- return new MRContainer();
- }
-
- public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
- while (result == null || !result.hasNext()) {
- String s = parser.slice();
- if (s == null)
- return false;
- result = parser.parse(s).iterator();
- };
- value.set((MRData)result.next());
- key.set(new MR_long(fsin.getPos()));
- return true;
- }
-
- public synchronized long getPos () throws IOException { return fsin.getPos(); }
-
- public synchronized void close () throws IOException { fsin.close(); }
-
- public float getProgress () throws IOException {
- if (end == start)
- return 0.0f;
- else return Math.min(1.0f, (getPos() - start) / (float)(end - start));
- }
- }
-
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split,
- JobConf job,
- Reporter reporter ) throws IOException {
- SparkEvaluator.load_source_dir(); // load the parsed source parameters from a file
- String path = ((FileSplit)split).getPath().toString();
- ParsedDataSource ds = (ParsedDataSource)DataSource.get(path,Plan.conf);
- return new ParsedRecordReader((FileSplit)split,job,ds.parser,(Trees)ds.args);
- }
-}
[18/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/TypeInference.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/TypeInference.gen b/core/src/main/java/org/apache/mrql/TypeInference.gen
new file mode 100644
index 0000000..5276ae1
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/TypeInference.gen
@@ -0,0 +1,1356 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.*;
+
+
+/** the type inference/checker for MRQL expressions and algebraic forms */
+public class TypeInference extends Translator {
+
+ private static Tree make_tuple ( Trees pl ) {
+ if (pl.length() == 1)
+ return pl.head();
+ return #<tuple(...pl)>;
+ }
+
+ public static Tree make_persistent_type ( Tree tp ) {
+ match tp {
+ case `f(...al):
+ Trees bs = #[];
+ for ( Tree a: al )
+ bs = bs.append(make_persistent_type(a));
+ String g = persistent_collection(f);
+ return #<`g(...bs)>;
+ };
+ return tp;
+ }
+
+ private static String max_collection ( String x, String y ) {
+ boolean is_bag = x.equals("Bag") || y.equals("Bag") || x.equals("bag") || y.equals("bag");
+ boolean is_persistent = x.equals("Bag") || y.equals("Bag") || x.equals("List") || y.equals("List");
+ return (is_bag) ? ((is_persistent) ? "Bag" : "bag") : ((is_persistent) ? "List" : "list");
+ }
+
+ private static boolean numerical ( String tp ) {
+ return #[short,int,long,float,double].member(#<`tp>);
+ }
+
+ static void type_error ( Tree e, String msg ) {
+ System.err.println("*** Type error (line: "+e.line+", position: "+e.position+")");
+ if (Config.trace && type_env.iterator().hasNext()) {
+ msg += "\nVariable Types:";
+ for ( String var: type_env )
+ msg += "\n "+var + ": " + print_type(type_env.lookup(var));
+ };
+ System.err.println("*** "+msg);
+ throw new Error("Type Error");
+ }
+
+ /** given that pattern has type tp, bind the pattern variables to types */
+ private static Trees bind_pattern_type ( Tree pattern, Tree tp ) {
+ Trees args = #[];
+ match pattern {
+ case tuple(...pl):
+ match tp {
+ case tuple(...tl):
+ if (tl.length() != pl.length())
+ type_error(pattern,"Tuple pattern "+print_query(pattern)+" must have exactly "
+ +tl.length()+" components");
+ int i = 0;
+ for ( Tree p: pl )
+ args = args.append(bind_pattern_type(p,tl.nth(i++)));
+ case `etp: type_error(pattern,"Wrong pattern: found "+print_query(pattern)
+ +" but expected a pattern that matches the type "+print_type(etp));
+ };
+ case record(...bl):
+ Trees attrs = #[];
+ match tp {
+ case record(...tl):
+ for ( Tree b: bl )
+ match b {
+ case bind(`n,`p):
+ boolean found = false;
+ if (attrs.member(n))
+ type_error(pattern,"Duplicate record attribute name: "+n
+ +" in pattern "+print_query(pattern));
+ attrs = attrs.append(n);
+ for ( Tree t: tl )
+ match t {
+ case bind(`nt,`pt):
+ if (!nt.equals(n))
+ fail;
+ found = true;
+ args = args.append(bind_pattern_type(p,pt));
+ };
+ if (!found)
+ type_error(pattern,"Wrong record component: "+n
+ +" in pattern "+print_query(pattern)
+ +" (expected one from "+print_type(tp)+")");
+ };
+ case `etp: type_error(pattern,"Wrong pattern: found "+print_query(pattern)
+ +" but expected a pattern that matches the type "+print_type(etp));
+ };
+ case typed(`p,`t):
+ if (subtype(t,tp))
+ args = bind_pattern_type(p,t);
+ else type_error(pattern,"Type "+print_type(t)+" in pattern "+print_query(pattern)
+ +" does not match the expected type "+print_type(tp));
+ case list(...pl):
+ match tp {
+ case list(`etp):
+ for ( Tree p: pl )
+ args = args.append(bind_pattern_type(p,etp));
+ case `stp: type_error(pattern,"List pattern "+print_query(pattern)
+ +" can only be used for lists (found "+print_type(tp)+")");
+ };
+ case call(`c,...s):
+ Tree ci = data_constructors.lookup(c.toString());
+ if (ci == null)
+ type_error(pattern,"Undefined data constructor "+c+" in pattern "+print_query(pattern));
+ match ci {
+ case `dname(`n,`dtp):
+ if (!subtype(tp,expand(#<`dname>)))
+ type_error(pattern,"Cannot use the data constructor "+print_query(pattern)
+ +" in a pattern that expects type "+print_type(tp));
+ args = args.append(bind_pattern_type(s.length() == 1 ? s.head() : #<tuple(...s)>,
+ dtp));
+ };
+ case any: ;
+ case `v:
+ if (v.is_variable()) {
+ args = args.append(v);
+ type_env.insert(v.toString(),tp);
+ } else if (!subtype(type_inference2(v),tp))
+ type_error(pattern,"The constant "+v+" in pattern "
+ +print_query(pattern)+" is not of type "+print_type(tp));
+ };
+ return args;
+ }
+
+ private static short needs_coerce ( Tree x, Tree y ) {
+ if (x.is_variable() && numerical(x.toString())
+ && y.is_variable() && numerical(y.toString())) {
+ short cx = MRContainer.type_code(x.toString());
+ short cy = MRContainer.type_code(y.toString());
+ return (cx == cy) ? -1 : (cx > cy) ? cx : cy;
+ };
+ return -1;
+ }
+
+ /** type equality in MRQL is structured equality, not named equality */
+ public static boolean equal_types ( Tree tx, Tree ty ) {
+ tx = expand(tx);
+ ty = expand(ty);
+ if (tx.equals(ty))
+ return true;
+ match tx {
+ case `f(...xs):
+ match ty {
+ case `g(...ys):
+ if (f.equals(g) && xs.length() == ys.length()) {
+ for ( ; !ys.is_empty(); xs = xs.tail(), ys = ys.tail() )
+ if (!equal_types(xs.head(),ys.head()))
+ return false;
+ return true;
+ }
+ }
+ };
+ return false;
+ }
+
+ /** is the collection type name S a subtype of that of T?
+ List \lt Bag \lt bag and List \lt list \lt bag
+ */
+ public static boolean subtype ( String S, String T ) {
+ return S.equals(T)
+ || (S.equals("List") && T.equals("list"))
+ || (S.equals("List") && T.equals("Bag"))
+ || (S.equals("list") && T.equals("bag"))
+ || (S.equals("Bag") && T.equals("bag"));
+ }
+
+ /** is the type tx a subtype of type ty? */
+ public static boolean subtype ( Tree tx, Tree ty ) {
+ tx = expand(tx);
+ ty = expand(ty);
+ if (ty.equals(#<any>))
+ return true;
+ if (equal_types(tx,ty))
+ return true;
+ if (tx.is_variable() && numerical(tx.toString())
+ && ty.is_variable() && numerical(ty.toString()))
+ return MRContainer.type_code(tx.toString()) <= MRContainer.type_code(ty.toString());
+ match tx {
+ case `T(`tex):
+ if (!is_collection(T))
+ fail;
+ match ty {
+ case `S(`tey):
+ if (is_collection(S))
+ return subtype(T,S) && subtype(tex,tey);
+ else fail
+ };
+ case union(...):
+ if (ty.equals(#<union>)) // used for XML functions
+ return true;
+ else fail
+ case `f(...xs):
+ match ty {
+ case `g(...ys):
+ if (f.equals(g))
+ return subtype(xs,ys);
+ }
+ };
+ return false;
+ }
+
+ public static boolean subtype ( Trees ts1, Trees ts2 ) {
+ if (ts1.length() != ts2.length())
+ return false;
+ for ( Trees s1 = ts1, s2 = ts2; !s1.is_empty(); s1=s1.tail(), s2=s2.tail() )
+ if (!subtype(s1.head(),s2.head()))
+ return false;
+ return true;
+ }
+
+ public static int compare_types ( Tree t1, Tree t2 ) {
+ if (t1.equals(#<any>) && t2.equals(#<any>))
+ return 0;
+ else if (t2.equals(#<any>))
+ return -1;
+ else if (t1.equals(#<any>))
+ return 1;
+ else if (t1 instanceof VariableLeaf && t2 instanceof VariableLeaf)
+ return MRContainer.type_code(((VariableLeaf)t1).value())
+ -MRContainer.type_code(((VariableLeaf)t2).value());
+ else if (t1 instanceof VariableLeaf)
+ return -1;
+ else return 1;
+ }
+
+ /** if the expression at loc.head has type tx that is a subtype of ty, coerce it to ty
+ * NOTE: it destructively changes the expression at loc
+ */
+ private static void coerce ( Tree tx, Tree ty, Trees loc ) {
+ tx = expand(tx);
+ ty = expand(ty);
+ if (ty.equals(#<any>) || equal_types(tx,ty))
+ return;
+ if (tx.is_variable() && numerical(tx.toString())
+ && ty.is_variable() && numerical(ty.toString())) {
+ loc.head = #<call(coerce,`(loc.head), // destructive
+ `(MRContainer.type_code(ty.toString())))>;
+ return;
+ };
+ match tx {
+ case `T(`tex):
+ if (!is_collection(T))
+ fail;
+ match ty {
+ case `S(`tey):
+ if (!is_collection(S))
+ fail;
+ if (is_persistent_collection(T) && !is_persistent_collection(S))
+ loc.head = #<Collect(`(loc.head))>; // destructive
+ if (subtype(tex,tey) && unify(tex,tey) == null) {
+ Tree nv = new_var();
+ Tree b = #<bag(`nv)>;
+ coerce(tex,tey,((Node)b).children);
+ loc.head = #<cmap(lambda(`nv,`b),`(loc.head))>; // destructive
+ return;
+ }
+ };
+ case tuple(...xs):
+ match ty {
+ case tuple(...ys):
+ Tree nv = new_var();
+ Trees nt = #[];
+ int i = 0;
+ for ( Trees xl = xs, yl = ys; xl != #[] && yl != #[]; xl = xl.tail, yl = yl.tail ) {
+ Trees dest = #[nth(`nv,`(i++))];
+ coerce(xl.head,yl.head,dest);
+ nt = nt.append(dest);
+ };
+ loc.head = #<let(`nv,`(loc.head),tuple(...nt))>; // destructive
+ }
+ case record(...xs):
+ match ty {
+ case record(...ys):
+ Tree nv = new_var();
+ Trees nt = #[];
+ for ( Tree x: xs )
+ match x {
+ case bind(`ax,`tex):
+ for ( Tree y: ys )
+ match y {
+ case bind(`ay,`tey):
+ if (equal_types(ax,ay)) {
+ Tree b = #<bind(`ax,project(`nv,`ax))>;
+ nt = nt.append(b);
+ coerce(tex,tey,((Node)b).children.tail);
+ }
+ }
+ };
+ loc.head = #<let(`nv,`(loc.head),record(...nt))>; // destructive
+ }
+ }
+ }
+
+ /** pure-mans type-unification (the any type unifies with everything) */
+ private static Trees unify ( Trees ts1, Trees ts2 ) {
+ Trees s = #[];
+ if (ts1.length() != ts2.length())
+ return null;
+ for ( Trees s1 = ts1, s2 = ts2; !s1.is_empty(); s1=s1.tail(), s2=s2.tail() ) {
+ Tree t = unify(s1.head(),s2.head());
+ if (t == null)
+ return null;
+ else s = s.append(t);
+ };
+ return s;
+ }
+
+ /** pure-mans type-unification (the any type unifies with everything) */
+ public static Tree unify ( Tree t1, Tree t2 ) {
+ t1 = expand(t1);
+ t2 = expand(t2);
+ if (t1.equals(#<any>))
+ return t2;
+ if (t2.equals(#<any>) || equal_types(t1,t2))
+ return t1;
+ match t1 {
+ case `T(`t):
+ match t2 {
+ case `S(`s):
+ if (!T.equals(S))
+ fail;
+ if (!is_collection(T))
+ fail;
+ Tree ts = unify(t,s);
+ if (ts != null)
+ return #<`T(`ts)>;
+ }
+ case `f(...ts1):
+ match t2 {
+ case `g(...ts2):
+ Trees s = unify(ts1,ts2);
+ if (f.equals(g) && s != null)
+ return #<`f(...s)>;
+ }
+ };
+ return null;
+ }
+
+ /** if the types tx and ty do not unify, try to coerce them */
+ static Tree unify ( Tree tx, Tree ty, Trees destx, Trees desty ) {
+ Tree tp = unify(tx,ty);
+ if (tp != null)
+ return tp;
+ else if (subtype(tx,ty)) {
+ coerce(tx,ty,destx);
+ return ty;
+ } else if (subtype(ty,tx)) {
+ coerce(ty,tx,desty);
+ return tx;
+ } else return null;
+ }
+
+ /** find a type from the list of types ts that is the supertype of all other types */
+ private static Tree maximum_type ( Trees ts ) {
+ Tree maxt = ts.head;
+ for ( Tree t: ts.tail )
+ if (subtype(maxt,t) || maxt.equals(#<any>))
+ maxt = t;
+ return maxt;
+ }
+
+ /** if the type tp is a named type, expand it using its definition */
+ public static Tree expand ( Tree tp ) {
+ if (!tp.is_variable())
+ return tp;
+ Tree rt = global_datatype_env.lookup(tp.toString());
+ if (rt != null)
+ return expand(rt);
+ rt = type_names.lookup(tp.toString());
+ if (rt == null)
+ return tp;
+ else return expand(rt);
+ }
+
+ /** infer the type of an expression and expand it if necessary
+ * @param e the expression
+ * @return the type of e
+ */
+ public static Tree type_inference2 ( Tree e ) {
+ return expand(type_inference(e));
+ }
+
+ /** infer the type of an expression
+ * @param e the expression
+ * @return the type of e
+ */
+ public static Tree type_inference ( Tree e ) {
+ match e {
+ case select(`opt_dist,`u,from(...bl),where(`c),groupby(...gs),orderby(...os)):
+ type_env.begin_scope();
+ Trees dvs = #[];
+ String max_type = "list";
+ for ( Tree b: bl )
+ match b {
+ case bind(`p,`d):
+ match type_inference2(d) {
+ case `T(`tp):
+ if (!is_collection(T))
+ fail;
+ dvs = dvs.append(bind_pattern_type(p,tp));
+ max_type = max_collection(T,max_type);
+ case `ftp:
+ type_error(e,"The from-binding domain "+print_query(d)+" in "
+ +print_query(e)+" must be a collection (found "
+ +print_type(ftp)+")");
+ }
+ };
+ Tree ctp = type_inference2(c);
+ if (unify(ctp,#<bool>) == null)
+ type_error(e,"The predicate "+print_query(c)+" in "+print_query(e)
+ +" must be a boolean (found "+print_type(ctp)+")");
+ match #<groupby(...gs)> {
+ case groupby(`h,...gl):
+ Trees pvs = #[];
+ for ( Tree g: gl )
+ match g {
+ case bind(`gp,`gd):
+ bind_pattern_type(gp,type_inference2(gd));
+ pvs = pvs.append(pattern_variables(gp));
+ };
+ // find the type of the partition variable
+ Trees partl = #[];
+ for ( Tree dv: pattern_variables(#<tuple(...dvs)>) )
+ partl = partl.append(#<bind(`dv,`(type_env.lookup(dv.toString())))>);
+ type_env.insert("partition",#<bag(record(...partl))>);
+ // lift domain variables to bags
+ for ( Tree dv: dvs )
+ if (!pvs.member(dv))
+ type_env.insert(dv.toString(),
+ #<bag(`(type_env.lookup(dv.toString())))>);
+ Tree htp = type_inference2(h);
+ if (unify(htp,#<bool>) == null)
+ type_error(e,"The group-by predicate "+print_query(h)+" in "+print_query(e)
+ +" must be a boolean (found "+print_type(htp)+")");
+ };
+ match #<orderby(...os)> {
+ case orderby(`l,...ol):
+ if (!l.equals(#<none>)) {
+ Tree ltp = type_inference2(l);
+ if (unify(ltp,#<int>) == null)
+ type_error(e,"The limit "+print_query(l)+" in "+print_query(e)
+ +" must be an int (found "+print_type(ltp)+")");
+ };
+ for ( Tree o: ol)
+ type_inference2(o);
+ Tree rtp = type_inference2(u);
+ type_env.end_scope();
+ return (is_persistent_collection(max_type)) ? #<List(`rtp)> : #<list(`rtp)>;
+ };
+ Tree rtp = type_inference2(u);
+ type_env.end_scope();
+ return (is_persistent_collection(max_type)) ? #<Bag(`rtp)> : #<bag(`rtp)>;
+ case select(`u,from(...bl),where(`c)):
+ String max_type = "list";
+ for ( Tree b: bl )
+ match b {
+ case bind(`v,`d):
+ match type_inference2(d) {
+ case `T(`tp):
+ if (!is_collection(T))
+ fail;
+ type_env.insert(v.toString(),tp);
+ max_type = max_collection(T,max_type);
+ case _: type_error(e,"Expected a collection: "+print_query(d)
+ +" in "+print_query(e));
+ }
+ };
+ if (unify(type_inference2(c),#<bool>) != null) {
+ Tree tp = type_inference(u);
+ return (is_persistent_collection(max_type)) ? #<Bag(`tp)> : #<bag(`tp)>;
+ } else type_error(e,"The select predicate must be boolean: "+print_query(e));
+ case let_bind(`p,`u,`b):
+ type_env.begin_scope();
+ bind_pattern_type(p,type_inference2(u));
+ Tree tp = type_inference2(b);
+ type_env.end_scope();
+ return tp;
+ case let(`p,`u,`b):
+ bind_pattern_type(p,type_inference2(u));
+ return type_inference2(b);
+ case case(`u,...cs):
+ Tree tp = type_inference2(u);
+ Trees ts = #[];
+ for ( Tree c: cs )
+ match c {
+ case case(`p,`b):
+ type_env.begin_scope();
+ bind_pattern_type(p,tp);
+ ts = ts.append(type_inference2(b));
+ type_env.end_scope();
+ };
+ Tree maxt = maximum_type(ts);
+ for ( ; cs != #[] && ts != #[]; cs = cs.tail, ts = ts.tail )
+ match cs.head {
+ case case(`p,`b):
+ if (subtype(ts.head,maxt))
+ coerce(ts.head,maxt,((Node)(cs.head)).children.tail);
+ else type_error(cs.head,"Mismatched case output: "+b);
+ };
+ return maxt;
+ case lambda(`v,`b):
+ if(!v.is_variable())
+ fail;
+ if (type_env.lookup(v.toString()) == null)
+ type_env.insert(v.toString(),#<any>);
+ return #<arrow(`(type_env.lookup(v.toString())),`(type_inference(b)))>;
+ case function(tuple(...params),`outp,`body):
+ Trees as = #[];
+ for ( Tree param: params )
+ match param {
+ case `bind(`v,`tp):
+ type_env.insert(v.toString(),tp);
+ as = as.append(tp);
+ };
+ Tree btp = type_inference(body);
+ if (!subtype(btp,outp))
+ type_error(e,"The type of the function body "+print_type(btp)
+ +" does not match the expected type "+print_type(outp)+"\n"+e);
+ if (unify(btp,outp) == null) { // the body needs coercion
+ Trees b = #[`body];
+ coerce(btp,outp,b);
+ body = b.head;
+ };
+ return #<arrow(tuple(...as),`outp)>;
+ case call(source,binary,`f,`tp):
+ return tp;
+ case call(source,binary,`f):
+ if (!f.is_string())
+ type_error(e,"The source file must be a constant string: "+print_query(f));
+ Tree tp = null;
+ if (Config.hadoop_mode)
+ tp = Plan.get_type(f.stringValue());
+ else tp = MapReduceAlgebra.get_type(f.stringValue());
+ if (tp == null)
+ type_error(e,"Cannot find the type of file "+f);
+ ((Node)e).children = ((Node)e).children.append(tp); // destructive
+ return tp;
+ case call(source,`parser,`f,...args):
+ if (!parser.is_variable())
+ type_error(e,"The parser must be a constant name: "+print_query(parser));
+ if (unify(type_inference(f),#<string>) == null)
+ type_error(e,"The source file must be a string: "+print_query(f));
+ try {
+ Class<? extends Parser> pc = DataSource.parserDirectory.get(parser.toString());
+ if (pc == null)
+ type_error(e,"Unrecognized parser: "+parser);
+ Parser p = pc.newInstance();
+ p.initialize(args);
+ return #<Bag(`(p.type()))>;
+ } catch (Exception x) {
+ type_error(e,"Unrecognized parser type: "+parser);
+ }
+ case typed(null,`tp):
+ return tp;
+ case typed(`f(...),`tp):
+ if (f.equals("tagged_union") || f.equals("union_value"))
+ return tp;
+ else fail
+ case typed(`x,`tp):
+ if (tp.is_variable() && !tp.equals(#<string>)
+ && MRContainer.type_code(tp.toString()) >= 0) {
+ Tree tx = type_inference(x);
+ if (tx.is_variable() && !tx.equals(#<string>)
+ && MRContainer.type_code(tx.toString()) >= 0)
+ return tp;
+ else type_error(e,"Expression "+print_query(x)+" of type "+print_type(tx)
+ +" cannot be coerced to type "+tp);
+ };
+ Tree tx = type_inference(x);
+ if (!subtype(tx,tp))
+ type_error(e,"Expression "+print_query(x)+" of type "+print_type(tx)
+ +" cannot be coerced to type "+print_type(tp));
+ if (unify(tx,tp) == null) // need to coerce x
+ coerce(tx,tp,((Node)e).children);
+ return tp;
+ case tuple(...el):
+ Trees s = #[];
+ for ( Tree a: el )
+ s = s.append(type_inference(a));
+ return #<tuple(...s)>;
+ case call(coerce,`x,`n):
+ return #<`(MRContainer.type_names[(int)n.longValue()])>;
+ case record(...el):
+ Trees s = #[];
+ Trees attrs = #[];
+ for ( Tree a: el )
+ match a {
+ case bind(`v,`b):
+ s = s.append(#<bind(`v,`(type_inference(b)))>);
+ if (attrs.member(v))
+ type_error(e,"Duplicate record attribute name: "+v+" in "+print_query(e));
+ attrs = attrs.append(v);
+ };
+ return #<record(...s)>;
+ case union_tag(`x):
+ return #<int>;
+ case `T(...el):
+ if (!is_collection(T))
+ fail;
+ if (el.is_empty())
+ return #<`T(any)>;
+ Trees ts = #[];
+ for ( Tree t: el )
+ ts = ts.append(type_inference(t));
+ Tree maxt = maximum_type(ts);
+ for ( ; el != #[] && ts != #[]; el = el.tail, ts = ts.tail )
+ if (subtype(ts.head,maxt))
+ coerce(ts.head,maxt,el);
+ else type_error(e,"Incompatible values in "+T+" construction: "+print_query(e));
+ return #<`T(`maxt)>;
+ case nth(`a,`n):
+ if (!n.is_long())
+ type_error(e,"Tuple index must be an integer: "+print_query(e));
+ int i = (int)n.longValue();
+ match type_inference2(a) {
+ case tuple(...ts):
+ if (i < 0 || i >= ts.length())
+ type_error(e,"Tuple index outside bounds: "+print_query(e));
+ return ts.nth(i);
+ case `S(tuple(...ts)):
+ if (!is_collection(S))
+ fail;
+ if (i < 0 || i >= ts.length())
+ type_error(e,"Tuple index outside bounds: "+print_query(e));
+ return #<`S(`(ts.nth(i)))>;
+ case `tp:
+ type_error(e,"Tuple index must be applied to a tuple: "
+ +print_query(a)+" of type: "+print_type(tp)+" in "+print_query(e));
+ };
+ case project(`a,`v): // highly overloaded
+ Tree ta = type_inference(a);
+ match ta {
+ case XML:
+ return #<list(XML)>;
+ case `S(`tp):
+ if (is_collection(S) && (tp.equals(#<XML>) || tp.equals(TopLevel.xml_type)))
+ return #<`S(XML)>;
+ };
+ if (ta.equals(TopLevel.xml_type))
+ return #<list(XML)>;
+ match expand(ta) {
+ case record(...ts):
+ for ( Tree t: ts )
+ match t {
+ case bind(`w,`tp):
+ if (equal_types(w,v))
+ return tp;
+ };
+ type_error(e,"Record "+print_query(a)+" does not have a component "+v);
+ case union(...tl):
+ Trees s = #[];
+ for ( Tree t: tl )
+ match t {
+ case `c(record(...ts)):
+ for ( Tree tn: ts )
+ match tn {
+ case bind(`w,`tp):
+ if (equal_types(w,v))
+ s = s.append(tp);
+ };
+ case `c(bag(tuple(string,`tv))):
+ s = s.append(tv);
+ };
+ if (s.length() == 0)
+ type_error(e,"Wrong record projection "+print_query(e)
+ + " over the union "+print_type(ta));
+ else if (s.length() > 1)
+ type_error(e,"Ambiguous record projection "+print_query(e)
+ + " over the union "+print_type(ta));
+ return s.head();
+ case `S(`ttp):
+ if (!is_collection(S))
+ fail;
+ match expand(ttp) {
+ case record(...ts):
+ for ( Tree t: ts )
+ match t {
+ case bind(`w,`tp):
+ if (equal_types(w,v))
+ return #<`S(`tp)>;
+ };
+ type_error(e,"The record collection "+print_query(a)
+ +" does not have a component "+v);
+ case tuple(string,`tv):
+ return tv;
+ case union(...tl):
+ Trees s = #[];
+ for ( Tree t: tl )
+ match t {
+ case `c(record(...ts)):
+ for ( Tree tn: ts )
+ match tn {
+ case bind(`w,`tp):
+ if (equal_types(w,v))
+ s = s.append(tp);
+ };
+ case `c(bag(tuple(string,`tv))):
+ s = s.append(tv);
+ };
+ if (s.length() == 0)
+ type_error(e,"Wrong record projection "+print_query(e)
+ + " over the union "+print_type(ta));
+ else if (s.length() > 1)
+ type_error(e,"Ambiguous record projection "+print_query(e)
+ + " over the union "+print_type(ta));
+ return #<`S(...s)>;
+ };
+ case `t:
+ type_error(e,"The record projection "+print_query(e)
+ + " cannot apply to the type "+print_type(t));
+ };
+ case index(`a,`i):
+ Tree ti = type_inference2(i);
+ match type_inference2(a) {
+ case bag(tuple(`tk,`tv)):
+ if (subtype(ti,tk)) {
+ coerce(ti,tk,((Node)e).children.tail);
+ return tv;
+ } else fail
+ case Bag(tuple(`tk,`tv)):
+ if (subtype(ti,tk)) {
+ coerce(ti,tk,((Node)e).children.tail);
+ return tv;
+ } else fail
+ case list(`tp):
+ if (unify(ti,#<int>) != null)
+ return tp;
+ else type_error(e,"List index must be an integer: "+print_query(e));
+ case List(`tp):
+ if (unify(ti,#<int>) != null)
+ return tp;
+ else type_error(e,"List index must be an integer: "+print_query(e));
+ case union(...tl):
+ Trees s = #[];
+ for ( Tree t: tl )
+ match expand(t) {
+ case `c(bag(tuple(`tk,`tv))):
+ if (unify(ti,tk) != null)
+ s = s.append(tv);
+ else fail
+ case `c(list(`tp)):
+ if (unify(ti,#<int>) != null)
+ s = s.append(tp);
+ else fail
+ };
+ if (s.length() == 0)
+ type_error(e,"Wrong indexing "+print_query(e)
+ + " in union "+print_type(#<union(...tl)>));
+ else if (s.length() > 1)
+ type_error(e,"Ambiguous indexing "+print_query(e)
+ + " in union "+print_type(#<union(...tl)>));
+ return s.head();
+ case `tp: type_error(e,"Indexing is not allowed for type "+print_type(tp)
+ +" with index "+print_type(ti)+" in "+print_query(e));
+ };
+ case range(`u,`i,`j):
+ if (unify(type_inference2(i),#<int>) == null
+ || unify(type_inference2(j),#<int>) == null)
+ type_error(e,"Range indexes must be integer expressions: "+print_query(e));
+ match type_inference2(u) {
+ case list(`a): return #<list(`a)>;
+ case List(`a): return #<list(`a)>;
+ };
+ type_error(e,"Range indexing must be applied to a list: "+print_query(e));
+ case range(`min,`max):
+ Tree tmin = type_inference(min);
+ Tree tmax = type_inference(max);
+ if (!subtype(tmin,#<long>))
+ type_error(e,"Expected a long integer for min: "+print_query(min));
+ else if (unify(tmin,#<long>) == null) // coerce
+ coerce(tmin,#<long>,((Node)e).children);
+ if (!subtype(tmax,#<long>))
+ type_error(e,"Expected a long integer for max: "+print_query(max));
+ else if (unify(tmax,#<long>) == null) // coerce
+ coerce(tmax,#<long>,((Node)e).children.tail);
+ return #<list(long)>;
+ case call(gen,`min,`max,`size):
+ return type_inference(#<gen(`min,`max,`size)>);
+ case gen(`min,`max,`size):
+ Tree tmin = type_inference(min);
+ Tree tmax = type_inference(max);
+ Tree tsize = type_inference(size);
+ if (!subtype(tmin,#<long>))
+ type_error(e,"Expected a long integer for min: "+print_query(min));
+ else if (unify(tmin,#<long>) == null) // coerce
+ coerce(tmin,#<long>,((Node)e).children);
+ if (!subtype(tmax,#<long>))
+ type_error(e,"Expected a long integer for max: "+print_query(max));
+ else if (unify(tmax,#<long>) == null) // coerce
+ coerce(tmax,#<long>,((Node)e).children.tail);
+ if (!subtype(tsize,#<long>))
+ type_error(e,"Expected a long integer for size: "+print_query(size));
+ else if (unify(tsize,#<long>) == null) // coerce
+ coerce(tsize,#<long>,((Node)e).children.tail.tail);
+ return #<Bag(long)>;
+ case dataset_size(`x):
+ return #<long>;
+ case groupBy(`u):
+ Tree tp = type_inference2(u);
+ match tp {
+ case `T(tuple(`k,`v)):
+ if (is_collection(T))
+ return (is_persistent_collection(T))
+ ? #<Bag(tuple(`k,list(`v)))>
+ : #<bag(tuple(`k,list(`v)))>;
+ };
+ type_error(e,"Wrong groupBy: "+print_query(e)+" of type "+print_type(tp));
+ case orderBy(`u):
+ Tree tp = type_inference2(u);
+ match tp {
+ case `T(tuple(`k,`v)):
+ if (is_collection(T))
+ return (is_persistent_collection(T))
+ ? #<List(tuple(`k,list(`v)))>
+ : #<list(tuple(`k,list(`v)))>;
+ };
+ type_error(e,"Wrong orderBy: "+print_query(e)+" of type "+print_type(tp));
+ case cmap(lambda(`v,`body),`s):
+ match type_inference2(s) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ type_env.insert(v.toString(),a);
+ match type_inference2(body) {
+ case `S(`tp):
+ if (!is_collection(S))
+ fail;
+ return #<`T(`tp)>;
+ case _: type_error(e,"cmap must return a collection: "+print_query(e));
+ };
+ case `t: type_error(e,"cmap must be over a collection: "+print_query(e)
+ +" (found type "+print_type(t)+")");
+ };
+ type_error(e,"Wrong cmap: "+print_query(e));
+ case fold(lambda(`v,`body),`n,`s):
+ match type_inference2(s) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ Tree tp = type_inference(n);
+ type_env.insert(v.toString(),#<tuple(`a,tp)>);
+ if (unify(type_inference2(body),tp) == null)
+ type_error(e,"Wrong types in fold: "+print_query(e));
+ return tp;
+ };
+ case join(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`vr,`br),`x,`y):
+ match type_inference2(x) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ match type_inference2(y) {
+ case `S(`b):
+ if (!is_collection(S))
+ fail;
+ type_env.insert(v1.toString(),a);
+ type_env.insert(v2.toString(),b);
+ T = transient_collection(T);
+ S = transient_collection(S);
+ type_env.insert(vr.toString(),#<tuple(`T(`a),`S(`b))>);
+ if (unify(type_inference2(b1),type_inference2(b2)) == null)
+ type_error(e,"Incompatible keys in join ("+type_inference2(b1)
+ +" and "+type_inference2(b2)+"): "+print_query(e));
+ match type_inference(br) {
+ case `S3(`ntp):
+ if (!is_collection(S3))
+ fail;
+ S3 = persistent_collection(S3);
+ return #<`S3(`ntp)>;
+ };
+ type_error(e,"The join reducer must return a collection: "+print_query(br));
+ case _: type_error(e,"The right join input is not a collection: "+print_query(y));
+ };
+ case _: type_error(e,"The left join input is not a collection: "+print_query(x));
+ };
+ case crossProduct(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`vr,`br),`x,`y):
+ match type_inference2(x) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ match type_inference2(y) {
+ case `S(`b):
+ if (!is_collection(S))
+ fail;
+ type_env.insert(v1.toString(),a);
+ type_env.insert(v2.toString(),b);
+ match type_inference2(b1) {
+ case `S1(`w1):
+ if (!is_collection(S1))
+ fail;
+ match type_inference2(b2) {
+ case `S2(`w2):
+ if (!is_collection(S2))
+ fail;
+ type_env.insert(vr.toString(),#<tuple(`w1,`w2)>);
+ match type_inference(br) {
+ case `S3(`ntp):
+ if (!is_collection(S3))
+ fail;
+ S3 = persistent_collection(S3);
+ return #<`S3(`ntp)>;
+ };
+ type_error(e,"The cross product reducer must return a collection: "+print_query(br));
+ case _: type_error(e,"Wrong right mapper in a cross product: "+print_query(b2));
+ };
+ case _: type_error(e,"Wrong left mapper in a cross product: "+print_query(b1));
+ };
+ case _: type_error(e,"The right cross product input is not a collection: "+print_query(y));
+ };
+ case _: type_error(e,"The left cross product input is not a collection: "+print_query(x));
+ };
+ case mapReduce(lambda(`vm,`bm),lambda(`vr,`br),`X,_):
+ match type_inference2(X) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ type_env.insert(vm.toString(),a);
+ match type_inference2(bm) {
+ case `S(tuple(`k,`b)):
+ if (!is_collection(S))
+ fail;
+ type_env.insert(vr.toString(),#<tuple(`k,list(`b))>);
+ match type_inference(br) {
+ case `S3(`ntp):
+ if (!is_collection(S3))
+ fail;
+ if (is_persistent_collection(T))
+ S3 = persistent_collection(S3);
+ return #<`S3(`ntp)>;
+ };
+ type_error(e,"The MapReduce reducer must return a collection: "+print_query(br));
+ case _: type_error(e,"Wrong mapper in mapReduce: "+print_query(bm));
+ }
+ };
+ type_error(e,"The MapReduce input is not a collection: "+print_query(X));
+ case mapReduce2(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`vr,`br),`X,`Y,_):
+ match type_inference2(X) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ match type_inference2(Y) {
+ case `S(`b):
+ if (!is_collection(S))
+ fail;
+ type_env.insert(v1.toString(),a);
+ type_env.insert(v2.toString(),b);
+ match type_inference2(b1) {
+ case `S1(tuple(`k1,`w1)):
+ if (!is_collection(S1))
+ fail;
+ match type_inference2(b2) {
+ case `S2(tuple(`k2,`w2)):
+ if (!is_collection(S2))
+ fail;
+ if (unify(k1,k2) == null)
+ type_error(e,"incompatible keys in mapReduce2: "+print_query(e));
+ S1 = transient_collection(S1);
+ S2 = transient_collection(S2);
+ type_env.insert(vr.toString(),#<tuple(`S1(`w1),`S2(`w2))>);
+ match type_inference(br) {
+ case `S3(`ntp):
+ if (!is_collection(S3))
+ fail;
+ if (is_persistent_collection(T) && is_persistent_collection(S))
+ S3 = persistent_collection(S3);
+ return #<`S3(`ntp)>;
+ };
+ type_error(e,"The MapReduce2 reducer must return a collection: "+print_query(br));
+ case _: type_error(e,"Wrong right mapper in mapReduce2: "+print_query(b2));
+ };
+ case _: type_error(e,"Wrong left mapper in mapReduce2: "+print_query(b1));
+ };
+ case _: type_error(e,"The right MapReduce2 input is not a collection: "+print_query(Y));
+ };
+ case _: type_error(e,"The left MapReduce2 input is not a collection: "+print_query(X));
+ };
+ case Collect(`x):
+ match type_inference2(x) {
+ case Bag(`tp): return #<bag(`tp)>;
+ case List(`tp): return #<list(`tp)>;
+ };
+ type_error(e,"You can only cache persistent collections: "+e);
+ case aggregate(lambda(`v,`b),`z,`X):
+ match type_inference2(X) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ Tree ztp = type_inference2(z);
+ type_env.insert(v.toString(),#<tuple(`ztp,`a)>);
+ if (!subtype(ztp,type_inference2(b)))
+ type_error(e,"Wrong accumulator: "+print_query(e));
+ return ztp;
+ };
+ type_error(e,"Aggregation input is not a collection: "+print_query(e));
+ case repeat(lambda(`p,`b),`s,...ns):
+ if (!ns.is_empty() && unify(type_inference2(ns.head()),#<int>) == null)
+ type_error(e,"The maximum number of steps in repeat must be an integer: "
+ +print_query(ns.head()));
+ Tree tp = type_inference2(s);
+ bind_pattern_type(p,tp);
+ match tp {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ match type_inference2(b) {
+ case `S(`c): // transitive closure
+ if (!is_collection(S))
+ fail;
+ if (unify(a,c) == null)
+ fail;
+ ((Node)e).name = #<closure>.toString(); // destructive
+ return #<`T(`a)>;
+ case `S(tuple(`w,`c)):
+ if (!is_collection(S))
+ fail;
+ if (unify(c,#<bool>) == null)
+ fail;
+ if (unify(a,w) == null)
+ fail;
+ return #<`T(`a)>;
+ case `t: type_error(e,"The repeat body must return a collection of type "
+ +print_type(tp)+" or "+print_type(#<`T(tuple(`a,bool))>)
+ +" (Found type: "+print_type(t)+")");
+ }
+ };
+ type_error(e,"The repeat source must return a bag: "+print_query(e));
+ case closure(lambda(`v,`b),`s,...ns):
+ if (!ns.is_empty() && unify(type_inference2(ns.head()),#<int>) == null)
+ type_error(e,"The maximum number of steps in closure must be an integer: "
+ +print_query(ns.head()));
+ match type_inference2(s) {
+ case `T(`a):
+ if (!is_collection(T))
+ fail;
+ type_env.insert(v.toString(),#<`T(`a)>);
+ match type_inference2(b) {
+ case `S(`tp):
+ if (!is_collection(S))
+ fail;
+ if (unify(a,tp) == null)
+ fail;
+ return #<`T(`a)>;
+ case `tp: type_error(e,"The closure body must return a collection of type "
+ +print_type(#<`T(`a)>)+" or "+print_type(#<`T(tuple(`a,bool))>)
+ +" (Found type: "+print_type(tp)+")");
+ }
+ };
+ type_error(e,"The closure source must return a bag: "+print_query(e));
+ case loop(lambda(`p,`b),`s,`n):
+ if (unify(type_inference2(n),#<int>) == null)
+ type_error(e,"The number of steps in loop must be an integer: "+print_query(n));
+ Tree tp = type_inference2(s);
+ bind_pattern_type(p,tp);
+ Tree btp = type_inference2(b);
+ if (unify(tp,btp) == null)
+ type_error(e,"The type of the repeat body ("+print_type(btp)
+ + ") and the type of the initial value ("+print_type(tp)+") do not match");
+ return tp;
+ case step(`x): // used in QueryPlan for repeat
+ match type_inference(x) {
+ case `T(`tp): return #<`T(tuple(`tp,bool))>;
+ };
+ case cstep(`x): // used in QueryPlan for closure
+ return type_inference(x);
+ case if(`p,`x,`y):
+ if (unify(type_inference2(p),#<bool>) == null)
+ type_error(e,"Expected a boolean predicate in if-then-else: "+print_query(p));
+ Tree tx = type_inference2(x);
+ Tree ty = type_inference2(y);
+ Tree rt = unify(tx,ty,((Node)e).children.tail,((Node)e).children.tail.tail);
+ if (rt == null)
+ type_error(e,"Incompatible types in if-then-else: "+print_query(e));
+ return rt;
+ case call(inv,`x):
+ return type_inference(x);
+ case call(plus,`x,`y):
+ Tree tx = type_inference2(x);
+ Tree ty = type_inference2(y);
+ match tx {
+ case `T(`xt):
+ if (!is_collection(T))
+ fail;
+ match ty {
+ case `S(`yt):
+ if (!is_collection(S))
+ fail;
+ Tree rt = unify(tx,ty,((Node)e).children.tail,((Node)e).children.tail.tail);
+ if (rt == null)
+ type_error(e,"Incompatible types in union/append: "+print_type(tx)+" and "+print_type(ty));
+ return rt;
+ }
+ };
+ fail
+ case `f(`x,`y):
+ if (! #[union,intersect,except].member(#<`f>))
+ fail;
+ Tree tx = type_inference2(x);
+ Tree ty = type_inference2(y);
+ match tx {
+ case `T(`t1):
+ if (!is_collection(T))
+ fail;
+ Tree t = unify(tx,ty,((Node)e).children,((Node)e).children.tail);
+ if (t != null)
+ return t;
+ };
+ type_error(e,"Incompatible types in "+f+": "+print_type(tx)+" and "+print_type(ty));
+ case call(member,`x,`y):
+ Tree tx = type_inference2(x);
+ Tree ty = type_inference2(y);
+ match ty {
+ case `T(`t1):
+ if (!is_collection(T))
+ fail;
+ if (!subtype(tx,t1))
+ type_error(e,"Incompatible types in member: "+print_type(tx)+" and "+print_type(ty));
+ coerce(tx,t1,((Node)e).children.tail);
+ return #<bool>;
+ };
+ case call(`f,`x,`y):
+ if (! #[eq,neq,lt,leq,gt,geq].member(f))
+ fail;
+ Tree tx = type_inference2(x);
+ Tree ty = type_inference2(y);
+ if (!subtype(tx,ty) && !subtype(ty,tx))
+ type_error(e,"Incompatible types in comparison "+f+": "
+ +print_type(tx)+" and "+print_type(ty));
+ if (unify(tx,ty) != null)
+ return #<bool>;
+ if (subtype(tx,ty))
+ coerce(tx,ty,((Node)e).children.tail);
+ else if (subtype(ty,tx))
+ coerce(ty,tx,((Node)e).children.tail.tail);
+ return #<bool>;
+ case call(`f,`s):
+ for ( Tree monoid: monoids )
+ match monoid {
+ case `aggr(`mtp,`plus,`zero,`unit):
+ if (!aggr.equals(f.toString()))
+ continue;
+ match type_inference2(s) {
+ case `T(`tp):
+ if (!is_collection(T))
+ type_error(e,"Aggregation must be over collections: "+s);
+ if (unify(mtp,tp) == null)
+ continue;
+ Tree nv = new_var();
+ type_env.begin_scope();
+ type_env.insert(nv.toString(),tp);
+ Tree t = type_inference2(#<apply(`unit,`nv)>);
+ Tree tz = type_inference2(zero);
+ type_env.end_scope();
+ if (unify(t,tz) != null)
+ return t;
+ }
+ };
+ fail
+ case call(avg,`s):
+ return type_inference(#<call(div,typed(call(sum,`s),double),call(count,`s))>);
+ case apply(lambda(`v,`b),`arg):
+ type_env.begin_scope();
+ type_env.insert(v.toString(),type_inference(arg));
+ Tree tp = type_inference(b);
+ type_env.end_scope();
+ return tp;
+ case call(`f,...al):
+ Tree macro = global_macros.lookup(f.toString());
+ if (macro == null)
+ fail;
+ match macro {
+ case macro(params(...pl),`body):
+ Tree b = body;
+ if (pl.length() != al.length())
+ fail;
+ for ( ; !pl.is_empty(); pl = pl.tail(), al = al.tail() )
+ b = subst(pl.head(),al.head(),b);
+ return type_inference2(b);
+ };
+ fail
+ case call(`f,...el):
+ if (!f.is_variable() || global_vars.lookup(f.toString()) != null)
+ fail;
+ Tree ret = data_constructors.lookup(f.toString());
+ if (ret != null)
+ match ret {
+ case `v(`n,`tp):
+ if (unify(type_inference(make_tuple(el)),tp) != null)
+ return #<`v>;
+ else type_error(e,"Wrong data construction: "+print_query(e));
+ };
+ ret = type_env.lookup(f.toString());
+ if (ret == null)
+ ret = global_type_env.lookup(f.toString());
+ if (ret != null)
+ match ret {
+ case arrow(`s,`d):
+ Tree tp = type_inference(#<tuple(...el)>);
+ if (!subtype(tp,s))
+ type_error(e,"The domain of the anonymous function "+print_type(s)+" in "+print_query(e)
+ +" does not match the argument types "+print_type(tp));
+ if (unify(tp,s) == null) // coerce args
+ match #<tuple(`tp,`s)> {
+ case tuple(tuple(...txs),tuple(...tys)):
+ for ( ; txs != #[]; txs = txs.tail, tys = tys.tail, el = el.tail )
+ coerce(txs.head,tys.head,el);
+ }
+ return d;
+ case _: type_error(e,"Expected a functional type for "+f+" (found "+print_type(ret)+")");
+ };
+ Trees tps = #[];
+ for ( Tree a: el )
+ tps = tps.append(type_inference(a));
+ for ( Tree fnc: functions )
+ match fnc {
+ case `fn(`otp,...tl):
+ if (!fn.equals(f.toString()) || !subtype(tps,tl))
+ fail;
+ if (f.equals(#<XMLchildren>))
+ return #<list(XML)>;
+ else if (f.equals(#<XMLattributes>))
+ return #<list(string)>;
+ else if (f.equals(#<XMLattribute>) && collection_type(otp))
+ return #<list(string)>;
+ else return otp;
+ };
+ type_error(e,"Undefined function "+f+" over arguments of type "+print_type(#<tuple(...tps)>));
+ case apply(`f,`u):
+ match type_inference2(f) {
+ case arrow(`s,`d):
+ Tree tp = type_inference(u);
+ if (!subtype(tp,s))
+ type_error(e,"The domain of the anonymous function "+print_type(s)+" in "+print_query(e)
+ +" does not match the argument types "+print_type(tp));
+ if (unify(tp,s) == null) // coerce args
+ coerce(tp,s,((Node)e).children.tail);
+ return d;
+ case `tp: type_error(e,"Expected a functional type for "+f+" (found "+print_type(tp)+")");
+ };
+ case callM(`f,_,...el):
+ return type_inference(#<call(`f,...el)>);
+ case true: return #<bool>;
+ case false: return #<bool>;
+ case null: return #<any>;
+ case `v:
+ if (!v.is_variable())
+ fail;
+ Tree ret1 = type_env.lookup(v.toString());
+ Tree ret2 = global_type_env.lookup(v.toString());
+ if (ret1 == null)
+ if (ret2 == null) {
+ Tree ret = global_vars.lookup(v.toString());
+ if (ret == null) {
+ String msg = "";
+ if (!Config.trace && type_env.iterator().hasNext()) {
+ msg += "\nVariable Types:";
+ for ( String var: type_env )
+ msg += "\n "+var + ": " + print_type(type_env.lookup(var));
+ };
+ type_error(e,"Undefined variable: "+v+msg);
+ } else if (!v.equals(ret))
+ return type_inference(ret);
+ } else return ret2;
+ else return ret1;
+ case `n:
+ if (n.is_long())
+ return #<int>;
+ else if (n.is_double())
+ return #<float>;
+ else if (n.is_string())
+ return #<string>;
+ };
+ type_error(e,"Wrong expression: "+print_query(e));
+ throw new Error();
+ }
+
+ /** check the type for inconsistencies and fix the transient/persistent components */
+ public static Tree normalize_type ( Tree type ) {
+ match type {
+ case record(...bs):
+ Trees as = #[];
+ Trees vs = #[];
+ for ( Tree b: bs )
+ match b {
+ case bind(`v,`tp):
+ if (v.is_variable())
+ if (vs.member(v))
+ type_error(type,"Duplicate record attributes: "+print_type(type));
+ else {
+ vs = vs.append(v);
+ as = as.append(#<bind(`v,`(normalize_type(tp)))>);
+ }
+ else type_error(type,"Expected an attribute name: "+v);
+ case _: type_error(type,"Ill-formed record type: "+print_type(type));
+ };
+ return #<record(...as)>;
+ case tuple(...ts):
+ Trees as = #[];
+ for ( Tree t: ts )
+ as = as.append(normalize_type(t));
+ return #<tuple(...as)>;
+ case union(...bs):
+ Trees as = #[];
+ for ( Tree b: bs )
+ match b {
+ case `c(`tp):
+ as = as.append(#<`c(`(normalize_type(tp)))>);
+ case _: type_error(type,"Ill-formed union type: "+print_type(type));
+ };
+ return #<union(...as)>;
+ case persistent(bag(`tp)):
+ Tree ntp = normalize_type(tp);
+ return #<Bag(`ntp)>;
+ case persistent(list(`tp)):
+ Tree ntp = normalize_type(tp);
+ return #<List(`ntp)>;
+ case `T(`tp):
+ if (is_collection(T))
+ return #<`T(`(normalize_type(tp)))>;
+ else fail
+ case `tp:
+ if (!tp.is_variable())
+ fail;
+ if (#[bool,byte,short,int,long,float,double,string].member(tp))
+ return tp;
+ Tree rt = global_datatype_env.lookup(tp.toString());
+ if (rt != null)
+ return tp;
+ rt = type_names.lookup(tp.toString());
+ if (rt != null)
+ return tp;
+ };
+ type_error(type,"Unrecognized type: "+print_type(type));
+ return type;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Union.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Union.java b/core/src/main/java/org/apache/mrql/Union.java
new file mode 100644
index 0000000..540b6d9
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Union.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** union values are tagged values, where tag is the descriminator */
+public class Union extends MRData {
+ private byte tag;
+ private MRData value;
+
+ public Union ( byte tag, MRData value ) {
+ this.tag = tag;
+ this.value = value;
+ }
+
+ public void materializeAll () { value.materializeAll(); };
+
+ public byte tag () { return tag; }
+
+ public MRData value () { return value; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.UNION);
+ out.writeByte(tag);
+ value.write(out);
+ }
+
+ final public static Union read ( DataInput in ) throws IOException {
+ return new Union(in.readByte(),MRContainer.read(in));
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ tag = in.readByte();
+ value = MRContainer.read(in);
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof Union);
+ Union p = (Union) x;
+ return (tag == p.tag)
+ ? value.compareTo(p.value)
+ : (tag - p.tag);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ int k = (x[xs] == y[ys]) ? MRContainer.compare(x,xs+1,xl-1,y,ys+1,yl-1,size) : (x[xs]-y[ys]);
+ size[0] += 2;
+ return k;
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof Union && ((Union)x).tag==tag
+ && ((Union)x).value.equals(value);
+ }
+
+ public int hashCode () {
+ return Math.abs(tag ^ value.hashCode());
+ }
+
+ public String toString () {
+ return "union("+tag+","+value+")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/XMLParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/XMLParser.java b/core/src/main/java/org/apache/mrql/XMLParser.java
new file mode 100644
index 0000000..a25acbd
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/XMLParser.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import javax.xml.parsers.SAXParserFactory;
+import org.xml.sax.*;
+import java.io.*;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** An XML parser */
+final public class XMLParser implements Parser {
+ String[] tags; // split tags
+ Tree xpath; // XPath query for fragmentation
+ XMLSplitter splitter;
+ XPathParser parser;
+ XMLReader xmlReader;
+ final static SAXParserFactory factory = SAXParserFactory.newInstance();
+
+ public void initialize ( Trees args ) {
+ try {
+ if (args.length() > 0) {
+ if (!(args.nth(0) instanceof Node)
+ || !(((Node)args.nth(0)).name().equals("list")
+ || ((Node)args.nth(0)).name().equals("bag")))
+ throw new Error("Expected a bag of synchronization tagnames to split the XML source: "+args.nth(0));
+ Trees ts = ((Node)args.nth(0)).children();
+ if (ts.length() == 0)
+ throw new Error("Expected at least one synchronization tagname in XML source: "+ts);
+ tags = new String[ts.length()];
+ for ( int i = 0; i < tags.length; i++ )
+ if (ts.nth(i) instanceof StringLeaf)
+ tags[i] = ((StringLeaf)(ts.nth(i))).value();
+ else throw new Error("Expected a synchronization tagname in XML source: "+ts.nth(i));
+ if (args.length() == 2)
+ xpath = ((Node)args.nth(1)).children().nth(0);
+ else xpath = new VariableLeaf("dot");
+ } else xpath = new VariableLeaf("dot");
+ parser = new XPathParser(xpath);
+ factory.setValidating(false);
+ factory.setNamespaceAware(false);
+ xmlReader = factory.newSAXParser().getXMLReader();
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public void open ( String file ) {
+ try {
+ splitter = new XMLSplitter(tags,file,new DataOutputBuffer());
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public void open ( FSDataInputStream fsin, long start, long end ) {
+ try {
+ splitter = new XMLSplitter(tags,fsin,start,end,new DataOutputBuffer());
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public Tree type () { return new VariableLeaf("XML"); }
+
+ public String slice () {
+ if (splitter.hasNext()) {
+ DataOutputBuffer b = splitter.next();
+ return new String(b.getData(),0,b.getLength());
+ } else return null;
+ }
+
+ public Bag parse ( String s ) {
+ try {
+ parser.dataConstructor.start();
+ xmlReader.setContentHandler(parser.handler);
+ xmlReader.parse(new InputSource(new StringReader(s)));
+ Bag b = new Bag();
+ for ( MRData e: parser.dataConstructor.value() )
+ b.add(e);
+ return b;
+ } catch (Exception e) {
+ System.err.println(e);
+ return new Bag();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/XMLSplitter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/XMLSplitter.java b/core/src/main/java/org/apache/mrql/XMLSplitter.java
new file mode 100644
index 0000000..e407d85
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/XMLSplitter.java
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.Iterator;
+import javax.xml.parsers.SAXParserFactory;
+import org.xml.sax.*;
+import java.io.*;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+
+/** Extract the XML elements tagged by tags from a data split of the input stream (fsin)
+ * and store them in a buffer (to be parsed by SAX).
+ */
+final public class XMLSplitter implements Iterator<DataOutputBuffer> {
+ boolean in_memory;
+ FSDataInputStream fsin; // for HDFS processing
+ BufferedReader in; // for in-memory processing
+ String[] tags;
+ long start;
+ long end;
+ StringBuffer tagname = new StringBuffer(100);
+ String start_tagname;
+ final DataOutputBuffer buffer;
+
+ XMLSplitter ( String[] tags, FSDataInputStream fsin, long start, long end,
+ DataOutputBuffer buffer ) {
+ in_memory = false;
+ this.tags = tags;
+ this.fsin = fsin;
+ this.start = start;
+ this.end = end;
+ this.buffer = buffer;
+ try {
+ fsin.seek(start);
+ } catch ( IOException e ) {
+ System.err.println("*** Cannot parse the data split: "+fsin);
+ }
+ }
+
+ XMLSplitter ( String[] tags, String file, DataOutputBuffer buffer ) {
+ in_memory = true;
+ try {
+ in = new BufferedReader(new InputStreamReader(new FileInputStream(file)),
+ 100000);
+ } catch ( Exception e ) {
+ throw new Error("Cannot open the file: "+file);
+ };
+ this.tags = tags;
+ this.buffer = buffer;
+ }
+
+ public boolean hasNext () {
+ try {
+ if (in_memory || fsin.getPos() < end)
+ if (skip())
+ return store();
+ return false;
+ } catch (Exception e) {
+ System.err.println(e);
+ return false;
+ }
+ }
+
+ public DataOutputBuffer next () {
+ return buffer;
+ }
+
+ public void remove () { }
+
+ boolean is_start_tag () {
+ if (tags == null)
+ return true;
+ for (String tag: tags)
+ if (tag.contentEquals(tagname))
+ return true;
+ return false;
+ }
+
+ char read_tag () throws IOException {
+ tagname.setLength(0);
+ while (true) {
+ int b = in_memory ? in.read() : fsin.read();
+ if (b == -1)
+ return ' ';
+ else if (!Character.isLetterOrDigit(b) && b != ':' && b != '_')
+ return (char)b;
+ tagname.append((char)b);
+ }
+ }
+
+ /** skip until the beginning of a split element */
+ boolean skip () throws IOException {
+ while (true) {
+ int b = in_memory ? in.read() : fsin.read();
+ if (b == -1 || (!in_memory && fsin.getPos() >= end))
+ return false;
+ else if (b == '<') {
+ b = read_tag();
+ if (is_start_tag()) {
+ buffer.reset();
+ buffer.write('<');
+ for ( int i = 0; i < tagname.length(); i++ )
+ buffer.write(tagname.charAt(i));
+ buffer.write(b);
+ start_tagname = new String(tagname);
+ return true;
+ }
+ }
+ }
+ }
+
+ /** store one split element into the buffer; may cross split boundaries */
+ boolean store () throws IOException {
+ while (true) {
+ int b = in_memory ? in.read() : fsin.read();
+ if (b == -1)
+ return false;
+ if (b == '&') { // don't validate external XML entities
+ buffer.write('&');buffer.write('a');buffer.write('m');buffer.write('p');buffer.write(';');
+ } else buffer.write(b);
+ if (b == '<') {
+ b = in_memory ? in.read() : fsin.read();
+ buffer.write(b);
+ if (b == '/') {
+ b = read_tag();
+ for ( int i = 0; i < tagname.length(); i++ )
+ buffer.write(tagname.charAt(i));
+ buffer.write(b);
+ if (start_tagname.contentEquals(tagname)) {
+ while (b != '>') {
+ b = fsin.read();
+ buffer.write(b);
+ };
+ return true;
+ }
+ }
+ }
+ }
+ }
+}
[05/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/SystemFunctions.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/SystemFunctions.java b/src/main/java/core/SystemFunctions.java
deleted file mode 100644
index 5c0ed4b..0000000
--- a/src/main/java/core/SystemFunctions.java
+++ /dev/null
@@ -1,467 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.util.*;
-import java.lang.Math;
-
-
-/** System functions must be from MRData to MRData */
-final public class SystemFunctions {
-
- // used for shortcutting sync in bsp supersteps
- final public static Bag bsp_empty_bag = new Bag();
- final public static MR_bool bsp_true_value = new MR_bool(true);
- final public static MR_bool bsp_false_value = new MR_bool(false);
-
- final static MRData null_value = new Tuple(0);
- final static MR_bool true_value = new MR_bool(true);
- final static MR_bool false_value = new MR_bool(false);
-
- private static void error ( String s ) {
- System.err.println("*** "+s);
- throw new Error(s);
- }
-
- static Random random = new Random();
-
- public static MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
- return Evaluator.evaluator.synchronize(peerName,mr_exit);
- }
-
- public static Bag distribute ( MR_string peerName, Bag s ) {
- return Evaluator.evaluator.distribute(peerName,s);
- }
-
- public static MR_bool lt ( MR_short x, MR_short y ) { return (x.get() < y.get()) ? true_value : false_value; }
- public static MR_bool lt ( MR_int x, MR_int y ) { return (x.get() < y.get()) ? true_value : false_value; }
- public static MR_bool lt ( MR_long x, MR_long y ) { return (x.get() < y.get()) ? true_value : false_value; }
- public static MR_bool lt ( MR_float x, MR_float y ) { return (x.get() < y.get()) ? true_value : false_value; }
- public static MR_bool lt ( MR_double x, MR_double y ) { return (x.get() < y.get()) ? true_value : false_value; }
-
- public static MR_bool gt ( MR_short x, MR_short y ) { return (x.get() > y.get()) ? true_value : false_value; }
- public static MR_bool gt ( MR_int x, MR_int y ) { return (x.get() > y.get()) ? true_value : false_value; }
- public static MR_bool gt ( MR_long x, MR_long y ) { return (x.get() > y.get()) ? true_value : false_value; }
- public static MR_bool gt ( MR_float x, MR_float y ) { return (x.get() > y.get()) ? true_value : false_value; }
- public static MR_bool gt ( MR_double x, MR_double y ) { return (x.get() > y.get()) ? true_value : false_value; }
-
- public static MR_bool leq ( MR_short x, MR_short y ) { return (x.get() <= y.get()) ? true_value : false_value; }
- public static MR_bool leq ( MR_int x, MR_int y ) { return (x.get() <= y.get()) ? true_value : false_value; }
- public static MR_bool leq ( MR_long x, MR_long y ) { return (x.get() <= y.get()) ? true_value : false_value; }
- public static MR_bool leq ( MR_float x, MR_float y ) { return (x.get() <= y.get()) ? true_value : false_value; }
- public static MR_bool leq ( MR_double x, MR_double y ) { return (x.get() <= y.get()) ? true_value : false_value; }
-
- public static MR_bool geq ( MR_short x, MR_short y ) { return (x.get() >= y.get()) ? true_value : false_value; }
- public static MR_bool geq ( MR_int x, MR_int y ) { return (x.get() >= y.get()) ? true_value : false_value; }
- public static MR_bool geq ( MR_long x, MR_long y ) { return (x.get() >= y.get()) ? true_value : false_value; }
- public static MR_bool geq ( MR_float x, MR_float y ) { return (x.get() >= y.get()) ? true_value : false_value; }
- public static MR_bool geq ( MR_double x, MR_double y ) { return (x.get() >= y.get()) ? true_value : false_value; }
-
- public static MR_bool eq ( MR_short x, MR_short y ) { return (x.get() == y.get()) ? true_value : false_value; }
- public static MR_bool eq ( MR_int x, MR_int y ) { return (x.get() == y.get()) ? true_value : false_value; }
- public static MR_bool eq ( MR_long x, MR_long y ) { return (x.get() == y.get()) ? true_value : false_value; }
- public static MR_bool eq ( MR_float x, MR_float y ) { return (x.get() == y.get()) ? true_value : false_value; }
- public static MR_bool eq ( MR_double x, MR_double y ) { return (x.get() == y.get()) ? true_value : false_value; }
-
- public static MR_bool neq ( MR_short x, MR_short y ) { return (x.get() != y.get()) ? true_value : false_value; }
- public static MR_bool neq ( MR_int x, MR_int y ) { return (x.get() != y.get()) ? true_value : false_value; }
- public static MR_bool neq ( MR_long x, MR_long y ) { return (x.get() != y.get()) ? true_value : false_value; }
- public static MR_bool neq ( MR_float x, MR_float y ) { return (x.get() != y.get()) ? true_value : false_value; }
- public static MR_bool neq ( MR_double x, MR_double y ) { return (x.get() != y.get()) ? true_value : false_value; }
-
- public static MR_bool eq ( MRData x, MRData y ) { return (x.compareTo(y) == 0) ? true_value : false_value; }
- public static MR_bool neq ( MRData x, MRData y ) { return (x.compareTo(y) != 0) ? true_value : false_value; }
- public static MR_bool lt ( MRData x, MRData y ) { return (x.compareTo(y) < 0) ? true_value : false_value; }
- public static MR_bool leq ( MRData x, MRData y ) { return (x.compareTo(y) <= 0) ? true_value : false_value; }
- public static MR_bool gt ( MRData x, MRData y ) { return (x.compareTo(y) > 0) ? true_value : false_value; }
- public static MR_bool geq ( MRData x, MRData y ) { return (x.compareTo(y) >= 0) ? true_value : false_value; }
-
- public static MR_string plus ( MR_string x, MR_string y ) { return new MR_string(x.get()+y.get()); }
-
- public static Bag plus ( Bag x, Bag y ) { return x.union(y); }
-
- public static MR_short plus ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()+y.get())); }
- public static MR_int plus ( MR_int x, MR_int y ) { return new MR_int(x.get()+y.get()); }
- public static MR_long plus ( MR_long x, MR_long y ) { return new MR_long(x.get()+y.get()); }
- public static MR_float plus ( MR_float x, MR_float y ) { return new MR_float(x.get()+y.get()); }
- public static MR_double plus ( MR_double x, MR_double y ) { return new MR_double(x.get()+y.get()); }
-
- public static MR_short minus ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()-y.get())); }
- public static MR_int minus ( MR_int x, MR_int y ) { return new MR_int(x.get()-y.get()); }
- public static MR_long minus ( MR_long x, MR_long y ) { return new MR_long(x.get()-y.get()); }
- public static MR_float minus ( MR_float x, MR_float y ) { return new MR_float(x.get()-y.get()); }
- public static MR_double minus ( MR_double x, MR_double y ) { return new MR_double(x.get()-y.get()); }
-
- public static MR_short times ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()*y.get())); }
- public static MR_int times ( MR_int x, MR_int y ) { return new MR_int(x.get()*y.get()); }
- public static MR_long times ( MR_long x, MR_long y ) { return new MR_long(x.get()*y.get()); }
- public static MR_float times ( MR_float x, MR_float y ) { return new MR_float(x.get()*y.get()); }
- public static MR_double times ( MR_double x, MR_double y ) { return new MR_double(x.get()*y.get()); }
-
- public static MR_short div ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()/y.get())); }
- public static MR_int div ( MR_int x, MR_int y ) { return new MR_int(x.get()/y.get()); }
- public static MR_long div ( MR_long x, MR_long y ) { return new MR_long(x.get()/y.get()); }
- public static MR_float div ( MR_float x, MR_float y ) { return new MR_float(x.get()/y.get()); }
- public static MR_double div ( MR_double x, MR_double y ) { return new MR_double(x.get()/y.get()); }
-
- public static MR_int mod ( MR_int x, MR_int y ) { return new MR_int(x.get() % y.get()); }
- public static MR_long mod ( MR_long x, MR_long y ) { return new MR_long(x.get() % y.get()); }
-
- public static MR_short min ( MR_short x, MR_short y ) { return new MR_short((short)(Math.min(x.get(),y.get()))); }
- public static MR_int min ( MR_int x, MR_int y ) { return new MR_int(Math.min(x.get(),y.get())); }
- public static MR_long min ( MR_long x, MR_long y ) { return new MR_long(Math.min(x.get(),y.get())); }
- public static MR_float min ( MR_float x, MR_float y ) { return new MR_float(Math.min(x.get(),y.get())); }
- public static MR_double min ( MR_double x, MR_double y ) { return new MR_double(Math.min(x.get(),y.get())); }
-
- public static MR_short max ( MR_short x, MR_short y ) { return new MR_short((short)(Math.max(x.get(),y.get()))); }
- public static MR_int max ( MR_int x, MR_int y ) { return new MR_int(Math.max(x.get(),y.get())); }
- public static MR_long max ( MR_long x, MR_long y ) { return new MR_long(Math.max(x.get(),y.get())); }
- public static MR_float max ( MR_float x, MR_float y ) { return new MR_float(Math.max(x.get(),y.get())); }
- public static MR_double max ( MR_double x, MR_double y ) { return new MR_double(Math.max(x.get(),y.get())); }
-
- public static MR_double sin ( MR_double x ) { return new MR_double(Math.sin(x.get())); }
- public static MR_double cos ( MR_double x ) { return new MR_double(Math.cos(x.get())); }
- public static MR_double tan ( MR_double x ) { return new MR_double(Math.tan(x.get())); }
- public static MR_double asin ( MR_double x ) { return new MR_double(Math.asin(x.get())); }
- public static MR_double acos ( MR_double x ) { return new MR_double(Math.acos(x.get())); }
- public static MR_double atan ( MR_double x ) { return new MR_double(Math.atan(x.get())); }
- public static MR_double pow ( MR_double x, MR_double y ) { return new MR_double(Math.pow(x.get(),y.get())); }
- public static MR_double sqrt ( MR_double x ) { return new MR_double(Math.sqrt(x.get())); }
- public static MR_double ceil ( MR_double x ) { return new MR_double(Math.ceil(x.get())); }
- public static MR_double floor ( MR_double x ) { return new MR_double(Math.floor(x.get())); }
- public static MR_double rint ( MR_double x ) { return new MR_double(Math.rint(x.get())); }
- public static MR_int round ( MR_float x ) { return new MR_int((int)Math.round(x.get())); }
- public static MR_long round ( MR_double x ) { return new MR_long(Math.round(x.get())); }
-
- public static MR_short plus ( MR_short x ) { return x; }
- public static MR_int plus ( MR_int x ) { return x; }
- public static MR_long plus ( MR_long x ) { return x; }
- public static MR_float plus ( MR_float x ) { return x; }
- public static MR_double plus ( MR_double x ) { return x; }
-
- public static MR_short minus ( MR_short x ) { return new MR_short((short)-x.get()); }
- public static MR_int minus ( MR_int x ) { return new MR_int(-x.get()); }
- public static MR_long minus ( MR_long x ) { return new MR_long(-x.get()); }
- public static MR_float minus ( MR_float x ) { return new MR_float(-x.get()); }
- public static MR_double minus ( MR_double x ) { return new MR_double(-x.get()); }
-
- public static MR_short abs ( MR_short x ) { return new MR_short((short) Math.abs(x.get())); }
- public static MR_int abs ( MR_int x ) { return new MR_int(Math.abs(x.get())); }
- public static MR_long abs ( MR_long x ) { return new MR_long(Math.abs(x.get())); }
- public static MR_float abs ( MR_float x ) { return new MR_float(Math.abs(x.get())); }
- public static MR_double abs ( MR_double x ) { return new MR_double(Math.abs(x.get())); }
-
- public static Inv inv ( MRData x ) { return new Inv(x); }
-
- public static MR_bool and ( MR_bool x, MR_bool y ) { return (x.get()) ? y : false_value; }
- public static MR_bool or ( MR_bool x, MR_bool y ) { return (x.get()) ? true_value : y; }
- public static MR_bool not ( MR_bool x ) { return (x.get()) ? false_value : true_value; }
-
- public static MR_bool toBool ( MR_string s ) { return (s.get().equals("true")) ? true_value : false_value; }
- public static MR_short toShort ( MR_string s ) { return new MR_short(Short.parseShort(s.get())); }
- public static MR_int toInt ( MR_string s ) { return new MR_int(Integer.parseInt(s.get())); }
- public static MR_long toLong ( MR_string s ) { return new MR_long(Long.parseLong(s.get())); }
- public static MR_float toFloat ( MR_string s ) { return new MR_float(Float.parseFloat(s.get())); }
- public static MR_double toDouble ( MR_string s ) { return new MR_double(Double.parseDouble(s.get())); }
-
- public static MR_int random ( MR_int n ) {
- int v = random.nextInt(n.get());
- return new MR_int(v);
- }
-
- public static MR_float log ( MR_float n ) { return new MR_float(Math.log(n.get())); }
- public static MR_double log ( MR_double n ) { return new MR_double(Math.log(n.get())); }
- public static MR_float exp ( MR_float n ) { return new MR_float(Math.exp(n.get())); }
- public static MR_double exp ( MR_double n ) { return new MR_double(Math.exp(n.get())); }
-
- public static MR_string string ( MRData x ) { return new MR_string(x.toString()); }
-
- public static MR_bool contains ( MR_string x, MR_string y ) { return new MR_bool(x.get().contains(y.get())); }
- public static MR_int length ( MR_string x ) { return new MR_int(x.get().length()); }
- public static MR_string substring ( MR_string x, MR_int b, MR_int e ) { return new MR_string(x.get().substring(b.get(),e.get())); }
-
- public static MR_bool exists ( Bag s ) {
- return (s.iterator().hasNext()) ? true_value : false_value;
- }
-
- public static MR_bool some ( Bag x ) {
- for ( MRData e: x )
- if (e instanceof MR_bool)
- if (((MR_bool)e).get())
- return true_value;
- return false_value;
- }
-
- public static MR_bool all ( Bag x ) {
- for ( MRData e: x )
- if (e instanceof MR_bool)
- if (!((MR_bool)e).get())
- return false_value;
- return true_value;
- }
-
- public static MR_bool member ( MRData e, Bag s ) {
- return (s.contains(e)) ? true_value : false_value;
- }
-
- public static MR_long count ( Bag s ) {
- if (s.materialized())
- return new MR_long(s.size());
- long i = 0;
- for ( MRData e: s )
- i++;
- return new MR_long(i);
- }
-
- public static MR_long hash_code ( MRData x ) {
- return new MR_long(x.hashCode());
- }
-
- public static MRData index ( Bag b, MR_int mi ) {
- int i = mi.get();
- if (i < 0)
- throw new Error("wrong index: "+i);
- if (b.materialized())
- return b.get(i);
- int k = 0;
- for ( MRData e: b )
- if (k++ == i)
- return e;
- throw new Error("wrong index: "+i);
- }
-
- public static Bag range ( Bag b, MR_int mi, MR_int mj ) {
- int i = mi.get();
- int j = mj.get();
- if (j < i)
- throw new Error("wrong range indexes: "+i+","+j);
- Bag bag = new Bag(j-i+1);
- int k = 0;
- for ( MRData e: b ) {
- if (k >= i && k <= j)
- bag.add(e);
- k++;
- };
- return bag;
- }
-
- public static Bag union ( Bag x, Bag y ) {
- return x.union(y);
- }
-
- public static Bag intersect ( Bag x, Bag y ) {
- x.materialize();
- Bag s = new Bag();
- for ( MRData e: y )
- if (x.contains(e))
- s.add(e);
- return s;
- }
-
- public static Bag except ( Bag x, Bag y ) {
- y.materialize();
- Bag s = new Bag();
- for ( MRData e: x )
- if (!y.contains(e))
- s.add(e);
- return s;
- }
-
- public static Bag materialize ( Bag x ) {
- x.materialize();
- return x;
- }
-
- /** coerce a basic type to a new type indicated by the basic type number */
- public static MRData coerce ( MRData from, MR_int type ) {
- byte tp = (byte)type.get();
- if (from instanceof MR_short) {
- if (tp == MRContainer.BYTE)
- return new MR_byte((byte)((MR_short)from).get());
- else if (tp == MRContainer.SHORT)
- return from;
- else if (tp == MRContainer.INT)
- return new MR_int((int)((MR_short)from).get());
- else if (tp == MRContainer.LONG)
- return new MR_long((long)((MR_short)from).get());
- else if (tp == MRContainer.FLOAT)
- return new MR_float((float)((MR_short)from).get());
- else if (tp == MRContainer.DOUBLE)
- return new MR_double((double)((MR_short)from).get());
- } else if (from instanceof MR_int) {
- if (tp == MRContainer.BYTE)
- return new MR_byte((byte)((MR_int)from).get());
- else if (tp == MRContainer.SHORT)
- return new MR_short((short)((MR_int)from).get());
- else if (tp == MRContainer.INT)
- return from;
- else if (tp == MRContainer.LONG)
- return new MR_long((long)((MR_int)from).get());
- else if (tp == MRContainer.FLOAT)
- return new MR_float((float)((MR_int)from).get());
- else if (tp == MRContainer.DOUBLE)
- return new MR_double((double)((MR_int)from).get());
- } else if (from instanceof MR_long) {
- if (tp == MRContainer.BYTE)
- return new MR_byte((byte)((MR_long)from).get());
- else if (tp == MRContainer.SHORT)
- return new MR_short((short)((MR_long)from).get());
- else if (tp == MRContainer.INT)
- return new MR_int((int)((MR_long)from).get());
- else if (tp == MRContainer.LONG)
- return from;
- else if (tp == MRContainer.FLOAT)
- return new MR_float((float)((MR_long)from).get());
- else if (tp == MRContainer.DOUBLE)
- return new MR_double((double)((MR_long)from).get());
- } else if (from instanceof MR_float) {
- if (tp == MRContainer.BYTE)
- return new MR_byte((byte)((MR_float)from).get());
- else if (tp == MRContainer.SHORT)
- return new MR_short((short)((MR_float)from).get());
- else if (tp == MRContainer.INT)
- return new MR_int((int)((MR_float)from).get());
- else if (tp == MRContainer.LONG)
- return new MR_long((long)((MR_float)from).get());
- if (tp == MRContainer.FLOAT)
- return from;
- else if (tp == MRContainer.DOUBLE)
- return new MR_double((double)((MR_float)from).get());
- } else if (from instanceof MR_double) {
- if (tp == MRContainer.BYTE)
- return new MR_byte((byte)((MR_double)from).get());
- else if (tp == MRContainer.SHORT)
- return new MR_short((short)((MR_double)from).get());
- else if (tp == MRContainer.INT)
- return new MR_int((int)((MR_double)from).get());
- else if (tp == MRContainer.LONG)
- return new MR_long((long)((MR_double)from).get());
- if (tp == MRContainer.FLOAT)
- return new MR_float((float)((MR_double)from).get());
- if (tp == MRContainer.DOUBLE)
- return from;
- };
- error("Cannot up-coerce the numerical value "+from);
- return null;
- }
-
- /** used in avg */
- public static MR_double avg_value ( MRData t ) {
- MR_double sum = (MR_double)((Tuple)t).first();
- MR_long count = (MR_long)((Tuple)t).second();
- return new MR_double(sum.get()/count.get());
- }
-
- public static MR_string text ( Union node ) {
- if (node.tag() == 1)
- return (MR_string)(node.value());
- Bag b = (Bag)((Tuple)node.value()).get(2);
- String s = "";
- for ( MRData e: b )
- if (((Union)e).tag() == 1)
- s += ((MR_string)(((Union)e).value())).get();
- return new MR_string(s);
- }
-
- public static MR_string text ( Bag nodes ) {
- MR_string b = new MR_string("");
- for ( MRData e: nodes )
- b = plus(b,text((Union)e));
- return b;
- }
-
- public static MR_string tag ( Union node ) {
- if (node.tag() == 1)
- error("Cannot extract the tagname of a CData: "+node);
- return (MR_string)((Tuple) node.value()).get(0);
- }
-
- public static MR_string XMLattribute ( MR_string tagname, Union node ) {
- if (node.tag() == 1)
- error("Element "+node+" does not have attributes");
- Tuple t = (Tuple)node.value();
- String tag = tagname.get();
- for ( MRData c: (Bag)t.get(1) ) {
- Tuple p = (Tuple)c;
- if (tag.equals(((MR_string)(p.get(0))).get()))
- return new MR_string(((MR_string)p.get(1)).get());
- };
- error("Element "+node+" does not have attribute "+tagname);
- return null;
- }
-
- public static Bag XMLattributes ( MR_string tagname, Union node ) {
- if (node.tag() == 1)
- return new Bag();
- Tuple t = (Tuple)node.value();
- Bag b = new Bag();
- String tag = tagname.get();
- for ( MRData c: (Bag)t.get(1) ) {
- Tuple p = (Tuple)c;
- if (tag.equals("*") || tag.equals(((MR_string)(p.get(0))).get()))
- b.add(p.get(1));
- };
- return b;
- }
-
- public static Bag XMLattributes ( MR_string tagname, Bag nodes ) {
- Bag b = new Bag();
- for ( MRData e: nodes )
- for (MRData c: XMLattributes(tagname,(Union)e))
- b.add(c);
- return b;
- }
-
- public static Bag XMLattribute ( MR_string tagname, Bag nodes ) {
- Bag b = new Bag();
- for ( MRData e: nodes )
- for (MRData c: XMLattributes(tagname,(Union)e))
- b.add(c);
- return b;
- }
-
- public static Bag XMLchildren ( MR_string tagname, Union node ) {
- if (node.tag() == 1)
- return new Bag();
- Tuple t = (Tuple)node.value();
- Bag b = new Bag();
- String tag = tagname.get();
- for ( MRData c: (Bag)t.get(2) )
- if (((Union)c).tag() == 0) {
- Tuple s = (Tuple)(((Union)c).value());
- if (tag.equals("*") || (((MR_string)(s.get(0))).get()).equals(tag))
- b.add(c);
- };
- return b;
- }
-
- public static Bag XMLchildren ( MR_string tagname, Bag nodes ) {
- Bag b = new Bag();
- for ( MRData e: nodes )
- for (MRData c: XMLchildren(tagname,(Union)e))
- b.add(c);
- return b;
- }
-
- public static MRData fold ( Lambda c, MRData z, Bag s ) {
- MRData v = z;
- for ( MRData e: s )
- z = c.lambda().eval(new Tuple(z,e));
- return z;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Test.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Test.java b/src/main/java/core/Test.java
deleted file mode 100644
index eca272c..0000000
--- a/src/main/java/core/Test.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.util.*;
-import org.apache.hadoop.conf.*;
-
-
-/** Test all the MRQL test queries */
-final public class Test {
- public static PrintStream print_stream;
- public static Configuration conf;
- static MRQLParser parser = new MRQLParser();
- static String result_directory;
- static PrintStream test_out;
- static PrintStream error_stream;
-
- private static int compare ( String file1, String file2 ) throws Exception {
- FileInputStream s1 = new FileInputStream(file1);
- FileInputStream s2 = new FileInputStream(file2);
- int b1, b2;
- int i = 1;
- while ((b1 = s1.read()) == (b2 = s2.read()) && b1 != -1 && b2 != -1)
- i++;
- return (b1 == -1 && b2 == -1) ? 0 : i;
- }
-
- private static void query ( File query ) throws Exception {
- String path = query.getPath();
- if (!path.endsWith(".mrql"))
- return;
- Translator.global_reset();
- String qname = query.getName();
- qname = qname.substring(0,qname.length()-5);
- test_out.print(" Testing "+qname+" ... ");
- String result_file = result_directory+"/"+qname+".txt";
- boolean exists = new File(result_file).exists();
- if (exists)
- System.setOut(new PrintStream(result_directory+"/result.txt"));
- else System.setOut(new PrintStream(result_file));
- try {
- parser = new MRQLParser(new MRQLLex(new FileInputStream(query)));
- Main.parser = parser;
- MRQLLex.reset();
- parser.parse();
- int i;
- if (exists && (i = compare(result_file,result_directory+"/result.txt")) > 0)
- test_out.println("MISMATCH AT "+(i-1));
- else if (exists)
- test_out.println("OK matched");
- else test_out.println("OK created");
- } catch (Error ex) {
- error_stream.println(qname+": "+ex);
- ex.printStackTrace(error_stream);
- test_out.println("FAILED");
- if (!exists)
- new File(result_file).delete();
- } catch (Exception ex) {
- error_stream.println(qname+": "+ex);
- ex.printStackTrace(error_stream);
- test_out.println("FAILED");
- if (!exists)
- new File(result_file).delete();
- } finally {
- if (Config.hadoop_mode)
- Plan.clean();
- if (Config.compile_functional_arguments)
- Compiler.clean();
- }
- }
-
- public static void main ( String[] args ) throws Exception {
- boolean hadoop = false;
- for ( String arg: args ) {
- hadoop |= arg.equals("-local") || arg.equals("-dist");
- Config.bsp_mode |= arg.equals("-bsp");
- Config.spark_mode |= arg.equals("-spark");
- };
- Config.map_reduce_mode = !Config.bsp_mode && !Config.spark_mode;
- if (Config.map_reduce_mode)
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.MapReduceEvaluator").newInstance();
- if (Config.bsp_mode)
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
- if (Config.spark_mode)
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.SparkEvaluator").newInstance();
- Config.quiet_execution = true;
- if (hadoop) {
- conf = Evaluator.evaluator.new_configuration();
- GenericOptionsParser gop = new GenericOptionsParser(conf,args);
- conf = gop.getConfiguration();
- args = gop.getRemainingArgs();
- };
- Config.parse_args(args,conf);
- Config.hadoop_mode = Config.local_mode || Config.distributed_mode;
- Evaluator.evaluator.init(conf);
- new TopLevel();
- Config.testing = true;
- if (Config.hadoop_mode && Config.bsp_mode)
- Config.write(Plan.conf);
- if (Main.query_file.equals("") || Config.extra_args.size() != 2)
- throw new Error("Must provide a query directory, a result directory, and an error log file");
- File query_dir = new File(Main.query_file);
- result_directory = Config.extra_args.get(0);
- (new File(result_directory)).mkdirs();
- error_stream = new PrintStream(Config.extra_args.get(1));
- System.setErr(error_stream);
- test_out = System.out;
- for ( File f: query_dir.listFiles() )
- query(f);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/TopLevel.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/TopLevel.gen b/src/main/java/core/TopLevel.gen
deleted file mode 100644
index 0ed3f49..0000000
--- a/src/main/java/core/TopLevel.gen
+++ /dev/null
@@ -1,293 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-
-
-/** Provides the API for compilation/code-generation */
-final public class TopLevel extends Interpreter {
- static Tree xml_type;
-
- public TopLevel () {
- // XML and JSON are user-defined types:
- datadef("XML",#<union(Node(tuple(string,bag(tuple(string,string)),list(XML))),
- CData(string))>);
- datadef("JSON",#<union(JObject(bag(tuple(string,JSON))),
- JArray(list(JSON)),
- Jstring(string),
- Jlong(long),
- Jdouble(double),
- Jbool(bool),
- Jnull(tuple()))>);
- constant(#<PI>,#<double>,new MR_double(Math.PI));
- xml_type = global_datatype_env.lookup("XML");
- DataSource.loadParsers();
- }
-
- private static boolean memory_parsed_source ( Tree e ) {
- match e {
- case ParsedSource(...): return true;
- case Merge(`x,`y): return memory_parsed_source(x) && memory_parsed_source(y);
- };
- return false;
- }
-
- /** translate and evaluate an MRQL expression into MRData
- * @param e MRQL query to be evaluated
- * @param print do we want to print the result?
- * @return the result of evaluation (MRData)
- */
- public static MRData expression ( Tree e, boolean print ) {
- try {
- Tree plan = translate_expression(e);
- query_plan = plan;
- tab_count = -3;
- if (plan == null)
- return null;
- if (Config.hadoop_mode)
- Evaluator.evaluator.initialize_query();
- MRData res = evalE(plan,null);
- if (print) {
- if (!Config.quiet_execution)
- System.out.println("Result:");
- if (!Config.hadoop_mode && Config.bsp_mode && memory_parsed_source(plan))
- System.out.println(print(((Tuple)((Bag)res).get(0)).second(),query_type));
- else System.out.println(print(res,query_type));
- } return res;
- } catch (Exception x) {
- if (x.getMessage() != null)
- System.err.println("*** MRQL System Error at line "+Main.parser.line_pos()+": "+x);
- if (Config.trace)
- x.printStackTrace(System.err);
- if (Config.testing)
- throw new Error(x);
- return null;
- } catch (Error x) {
- if (x.getMessage() != null)
- System.err.println("*** MRQL System Error at line "+Main.parser.line_pos()+": "+x);
- if (Config.trace)
- x.printStackTrace(System.err);
- if (Config.testing)
- throw new Error(x);
- return null;
- }
- }
-
- /** translate, evaluate, and print the results of an MRQL expression e
- * @param e MRQL query to be evaluated
- * @return the result of evaluation (MRData)
- */
- public final static MRData expression ( Tree e ) {
- reset();
- return expression(e,true);
- }
-
- /** handle the assignment v=e */
- public final static void assign ( String v, Tree e ) {
- if (variable_lookup(v,global_env) != null) {
- global_type_env.remove(v);
- remove_global_binding(v);
- };
- global_vars.insert(v,e);
- }
-
- private final static boolean is_function ( Tree e ) {
- match e {
- case function(...): return true;
- };
- return false;
- }
-
- /** handle the assignment v:=e */
- public final static Tree store ( String v, Tree e ) {
- reset();
- if (global_vars.lookup(v) != null)
- global_vars.remove(v);
- MRData res = expression(e,false);
- global_type_env.insert(v,query_type);
- if (res instanceof Bag)
- ((Bag)res).materialize();
- new_global_binding(v,res);
- return query_plan;
- }
-
- /** define an MRQL constant, such as PI */
- private final static void constant ( Tree v, Tree type, MRData value ) {
- String var = v.toString();
- if (global_vars.lookup(var) != null)
- global_vars.remove(var);
- global_type_env.insert(var,type);
- new_global_binding(var,value);
- }
-
- /** define a new function
- * @param fnc function name
- * @param params parameter list
- * @param out_type output type
- * @param body function body
- */
- public final static void functiondef ( String fnc, Trees params, Tree out_type, Tree body ) {
- reset();
- Trees as = #[];
- Trees ps = #[];
- for ( Tree param: params )
- match param {
- case bind(`v,`tp):
- Tree ntp = normalize_type(tp);
- as = as.append(ntp);
- ps = ps.append(#<bind(`v,`ntp)>);
- case _: type_error(param,"Ill-formed function parameter: "+param);
- };
- out_type = normalize_type(out_type);
- // needed for recursive functions
- global_type_env.insert(fnc,#<arrow(tuple(...as),`out_type)>);
- Tree fname = #<`fnc>;
- if (!is_pure(body))
- impure_functions = impure_functions.append(fname);
- Tree plan = store(fnc,#<function(tuple(...ps),`out_type,`body)>);
- if (plan != null)
- Translator.global_functions.insert(fnc,plan);
- if (Config.hadoop_mode && plan != null)
- Plan.conf.set("mrql.global."+fnc,
- closure(plan,global_env).toString());
- }
-
- /** dump the result of evaluating the MRQL query e to a binary file */
- private final static void dump ( String file, Tree e ) {
- MRData res = expression(e,false);
- try {
- query_type = make_persistent_type(query_type);
- if (res != null)
- if (Config.hadoop_mode)
- Evaluator.evaluator.dump(file,query_type,res);
- else MapReduceAlgebra.dump(file,query_type,res);
- } catch (Exception x) {
- throw new Error(x);
- }
- }
-
- /** dump the result of evaluating the MRQL query e to a text CVS file */
- private final static void dump_text ( String file, Tree e ) {
- MRData res = expression(e,false);
- if (res != null)
- try {
- Evaluator.evaluator.dump_text(file,query_type,res);
- } catch (Exception x) {
- throw new Error(x);
- }
- }
-
- /** define a new named type (typedef) */
- private final static void typedef ( String name, Tree type ) {
- type_names.insert(name,normalize_type(type));
- }
-
- /** define a new data type, such as XML and JSON */
- private final static void datadef ( String name, Tree type ) {
- int i = 0;
- Trees as = #[];
- match type {
- case union(...nl):
- // needed for recursive datatypes
- global_datatype_env.insert(name,#<union(...nl)>);
- for ( Tree n: nl )
- match n {
- case `c(`t):
- if (data_constructors.lookup(c.toString()) == null)
- data_constructors.insert(c.toString(),#<`name(`i,`t)>);
- else type_error(type,"Data constructor "+c+" has already been defined");
- as = as.append(#<`c(`(normalize_type(t)))>);
- i++;
- }
- };
- global_datatype_env.remove(name);
- global_datatype_env.insert(name,#<union(...as)>);
- }
-
- /** define a user aggregation */
- private static void aggregation ( String name, Tree type, Tree plus, Tree zero, Tree unit ) {
- reset();
- zero = Simplification.rename(zero);
- plus = Simplification.rename(plus);
- unit = Simplification.rename(unit);
- type = normalize_type(type);
- Tree ztp = TypeInference.type_inference2(zero);
- Tree v1 = new_var();
- type_env.insert(v1.toString(),ztp);
- TypeInference.type_inference2(Normalization.normalize_all(#<apply(`plus,tuple(`v1,`v1))>));
- Tree v2 = new_var();
- type_env.insert(v2.toString(),type);
- Tree utp = TypeInference.type_inference2(Normalization.normalize_all(#<apply(`unit,`v2)>));
- if (unify(utp,ztp) == null)
- type_error(unit,"Wrong type in unit result (expected "+ztp+" found "+utp);
- monoids = monoids.append(#<`name(`type,`plus,`zero,`unit)>);
- }
-
- /** the MRQL top-level interfacse to evaluate a single MRQL expression or command */
- public final static void evaluate_top_level ( Tree expr ) {
- if (expr == null)
- return;
- match expr {
- case expression(`e):
- long t = System.currentTimeMillis();
- if (expression(e) != null && !Config.quiet_execution)
- System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
- case assign(`v,`e): assign(v.toString(),e);
- case store(`v,`e):
- long t = System.currentTimeMillis();
- if (store(v.toString(),e) != null && !Config.quiet_execution)
- System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
- case dump(`s,`e):
- long t = System.currentTimeMillis();
- dump(s.stringValue(),e);
- if (!Config.quiet_execution)
- System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
- case dump_text(`s,`e):
- long t = System.currentTimeMillis();
- dump_text(s.stringValue(),e);
- if (!Config.quiet_execution)
- System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
- case typedef(`v,`t): typedef(v.toString(),t);
- case datadef(`v,`t): datadef(v.toString(),t);
- case functiondef(`f,params(...p),`tp,`e):
- functiondef(f.toString(),p,tp,e);
- case macrodef(`name,params(...p),`e):
- Translator.global_macros.insert(name.toString(),#<macro(params(...p),`e)>);
- case aggregation(`aggr,`type,`plus,`zero,`unit):
- aggregation(aggr.toString(),type,plus,zero,unit);
- case import(`c):
- ClassImporter.importClass(c.variableValue());
- case import(`c,...l):
- for (Tree m: l)
- ClassImporter.importMethod(c.variableValue(),m.variableValue());
- case include(`file):
- Main.include_file(file.toString());
- case parser(`n,`p):
- try {
- Class<? extends Parser> c = Class.forName(p.toString()).asSubclass(Parser.class);
- DataSource.parserDirectory.put(n.toString(),c);
- } catch (ClassNotFoundException e) {
- throw new Error("Class "+p.toString()+" not found");
- }
- case impure(`fn): // not used
- impure_functions = impure_functions.append(fn);
- case _: throw new Error("Unknown statement: "+expr);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Translator.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Translator.gen b/src/main/java/core/Translator.gen
deleted file mode 100644
index 47b34ac..0000000
--- a/src/main/java/core/Translator.gen
+++ /dev/null
@@ -1,445 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.*;
-import java.io.*;
-
-
-/** contains useful methods used by all stages of compilation and code generation */
-public class Translator extends Printer {
- static Trees functions = #[ ];
-
- static {
- ClassImporter.load_classes();
- DataSource.loadParsers();
- }
-
- /** type environment that binds local variables to types */
- static SymbolTable type_env = new SymbolTable();
-
- /** type environment that binds global variables to types */
- static SymbolTable global_type_env = new SymbolTable();
-
- /** type environment that binds datatype names to types */
- static SymbolTable global_datatype_env = new SymbolTable();
-
- /** used in pattern compilation and variable renaming */
- static SymbolTable st = new SymbolTable();
-
- /** binds macro names to MRQL expressions (used for 'name = expr;' syntax) */
- static SymbolTable global_vars = new SymbolTable();
-
- /** binds a UDF name to its plan */
- static SymbolTable global_functions = new SymbolTable();
-
- /** binds a macro name to its body */
- static SymbolTable global_macros = new SymbolTable();
-
- /** used in typedefs */
- static SymbolTable type_names = new SymbolTable();
-
- /** binds a data constructor name to its type */
- static SymbolTable data_constructors = new SymbolTable();
-
- static Trees repeat_variables = #[];
-
- static {
- global_type_env.insert("args",#<list(string)>);
- }
-
- /** expressions with impure functions cannot factored out */
- static Trees impure_functions = #[random];
-
- private static int var_count = 0;
-
- static void reset () {
- var_count = 0;
- type_env = global_type_env;
- st = new SymbolTable();
- type_env = new SymbolTable();
- repeat_variables = #[];
- }
-
- static void global_reset () {
- reset();
- global_type_env = new SymbolTable();
- global_datatype_env = new SymbolTable();
- global_vars = new SymbolTable();
- global_functions = new SymbolTable();
- global_macros = new SymbolTable();
- type_names = new SymbolTable();
- data_constructors = new SymbolTable();
- new TopLevel();
- }
-
- static void error ( String msg ) {
- System.err.println("*** MRQL error at line "+Main.parser.line_pos()+": "+msg);
- throw new Error();
- }
-
- final static Tree identity = #<lambda(x,x)>;
-
- static Tree identity () {
- return Normalization.rename(#<lambda(x,bag(x))>);
- }
-
- /** is this type a collection type? */
- public static boolean is_collection ( String x ) {
- return x.equals("Bag") || x.equals("bag") || x.equals("List") || x.equals("list");
- }
-
- /** is this type a collection type? */
- public static boolean collection_type ( Tree tp ) {
- match tp {
- case `T(`t1): return is_collection(T);
- };
- return false;
- }
-
- /** is this type a collection type for values stored in HDFS? */
- public static boolean is_persistent_collection ( String x ) {
- return x.equals("Bag") || x.equals("List");
- }
-
- /** make this collection type a persistent type that is stored in HDFS */
- public static String persistent_collection ( String x ) {
- return (x.equals("list")) ? "List" : (x.equals("bag")) ? "Bag" : x;
- }
-
- /** make this collection type a transient type stored in memory */
- public static String transient_collection ( String x ) {
- return (x.equals("List")) ? "list" : (x.equals("Bag")) ? "bag" : x;
- }
-
- /** An aggeregation must be based on a commutative monoid (plus,zero) with a unit:
- * name(type,plus,zero,unit)
- * plus: function from (b,b) to b, zero: b, unit: function from a to b
- */
- static Trees monoids =
- #[ count(any,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0,long),lambda(x,typed(1,long))),
- sum(int,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0,long),lambda(x,typed(x,long))),
- sum(long,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0,long),`identity),
- sum(float,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0.0,double),lambda(x,typed(x,double))),
- sum(double,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0.0,double),`identity),
- max(int,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Integer.MIN_VALUE),int),`identity),
- max(long,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Long.MIN_VALUE),long),`identity),
- max(float,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Float.MIN_VALUE),float),`identity),
- max(double,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Double.MIN_VALUE),double),`identity),
- min(int,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Integer.MAX_VALUE),int),`identity),
- min(long,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Long.MAX_VALUE),long),`identity),
- min(float,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Float.MAX_VALUE),float),`identity),
- min(double,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Double.MAX_VALUE),double),`identity),
- avg_aggr(int,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
- call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
- tuple(typed(0.0,double),typed(0,long)),
- lambda(x,tuple(typed(x,double),typed(1,long)))),
- avg_aggr(long,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
- call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
- tuple(typed(0.0,double),typed(0,long)),
- lambda(x,tuple(typed(x,double),typed(1,long)))),
- avg_aggr(float,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
- call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
- tuple(typed(0.0,double),typed(0,long)),
- lambda(x,tuple(typed(x,double),typed(1,long)))),
- avg_aggr(double,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
- call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
- tuple(typed(0.0,double),typed(0,long)),
- lambda(x,tuple(typed(x,double),typed(1,long)))),
- all(bool,lambda(x,call(and,nth(x,0),nth(x,1))),true,`identity),
- some(bool,lambda(x,call(or,nth(x,0),nth(x,1))),false,`identity)
- ];
-
- static void print_aggregates () {
- for ( Tree m: monoids )
- match m {
- case `f(`tp,...):
- System.out.print(" "+f+":"+print_type(tp));
- }
- System.out.println();
- }
-
- static Trees plans_with_distributed_lambdas
- = #[MapReduce,MapAggregateReduce,MapCombineReduce,FroupByJoin,Aggregate,
- MapReduce2,MapCombineReduce2,MapAggregateReduce2,MapJoin,MapAggregateJoin,
- CrossProduct,CrossAggregateProduct,cMap,AggregateMap,BSP,GroupByJoin,repeat,closure];
-
- static Trees algebraic_operators
- = #[mapReduce,mapReduce2,cmap,join,groupBy,orderBy,aggregate,map,filter];
-
- static Trees plan_names = plans_with_distributed_lambdas.append(algebraic_operators)
- .append(#[Repeat,Closure,Generator,Let,If]);
-
- /** generates new variable names */
- public static Tree new_var () {
- return new VariableLeaf("x_"+(Integer.toString(var_count++)));
- }
-
- /** is this expression pure? (does it contain calls to impure functions?) */
- static boolean is_pure ( Tree expr ) {
- match expr {
- case call(`f,...al):
- if (impure_functions.member(f))
- return false;
- else fail
- case `f(...al):
- for ( Tree a: al )
- if (!is_pure(a))
- return false;
- };
- return true;
- }
-
- public static Trees union ( Trees xs, Trees ys ) {
- Trees res = xs;
- for ( Tree y: ys )
- if (!xs.member(y))
- res = res.append(y);
- return res;
- }
-
- /** return the variables of a pattern */
- static Trees pattern_variables ( Tree pattern ) {
- Trees args = #[];
- match pattern {
- case tuple(...pl):
- for ( Tree p: pl )
- args = union(args,pattern_variables(p));
- case record(...bl):
- for ( Tree b: bl )
- match b {
- case bind(`n,`p):
- args = union(args,pattern_variables(p));
- };
- case typed(`p,_):
- args = pattern_variables(p);
- case `v:
- if (v.is_variable())
- args = #[`v];
- };
- return args;
- }
-
- /** replace all occurences of from_expr in expr with to_expr
- * @param from_expr target
- * @param to_expr replacement
- * @param expr input
- * @return equal to expr but with all occurences of from_expr replaced with to_expr
- */
- public static Tree subst ( Tree from_expr, Tree to_expr, Tree expr ) {
- if (expr.equals(from_expr))
- return to_expr;
- match expr {
- case lambda(`v,_):
- if (pattern_variables(v).member(from_expr))
- return expr;
- else fail
- case bind(`a,`u):
- return #<bind(`a,`(subst(from_expr,to_expr,u)))>;
- case `f(...al):
- return #<`f(...(subst_list(from_expr,to_expr,al)))>;
- };
- return expr;
- }
-
- /** replace all occurences of from_expr in el with to_expr
- * @param from_expr target
- * @param to_expr replacement
- * @param el list of input expressions
- * @return equal to el but with all occurences of from_expr replaced with to_expr
- */
- public static Trees subst_list ( Tree from_expr, Tree to_expr, Trees el ) {
- Trees bl = #[];
- for ( Tree e: el )
- bl = bl.append(subst(from_expr,to_expr,e));
- return bl;
- }
-
- /** replace all occurences of var in expr with to_expr only if to_expr is pure or it is used once only
- * @param var target
- * @param to_expr replacement
- * @param expr input
- * @return equal to expr but with all occurences of from_expr replaced with to_expr
- */
- public static Tree subst_var ( Tree var, Tree to_expr, Tree expr ) {
- if (!is_pure(to_expr) && occurences(var,expr) > 1)
- return #<let(`var,`to_expr,`expr)>;
- else return subst(var,to_expr,expr);
- }
-
- /** used in the MRQL parser to handle templates */
- public static Tree template ( Tree s ) {
- match s {
- case template(`parser,...as):
- try {
- Trees args = #[];
- String tx = "";
- int i = 0;
- for ( Tree a: as )
- match a {
- case text(`t): tx += t;
- case _: args = args.append(a);
- tx += "{{"+(i++)+"}}";
- };
- Class<? extends Parser> pc = DataSource.parserDirectory.get(parser.toString());
- if (pc == null)
- throw new Error("Unrecognized parser: "+parser);
- Parser p = pc.newInstance();
- p.initialize(#[]);
- Bag e = p.parse(tx);
- Tree res = Interpreter.reify(e.get(0),p.type());
- for ( int j = 0; j < i; j++ )
- res = subst(new VariableLeaf("t_"+j),args.nth(j),res);
- return res;
- } catch (Exception e) {
- throw new Error("Wrong template: "+s+"\n"+e);
- }
- };
- throw new Error("Wrong template: "+s);
- }
-
- /** convert Tree constructions to code that construct these Trees (used in the Compiler) */
- public static String reify ( Tree e ) {
- if (e instanceof LongLeaf)
- return "new org.apache.mrql.gen.LongLeaf(" + e + ")";
- else if (e instanceof DoubleLeaf)
- return "new org.apache.mrql.gen.DoubleLeaf(" + e + ")";
- else if (e instanceof VariableLeaf)
- return "new org.apache.mrql.gen.VariableLeaf(\"" + e.variableValue() + "\")";
- else if (e instanceof StringLeaf)
- return "new org.apache.mrql.gen.StringLeaf(" + e.toString().replace("\\","\\\\") + ")";
- else {
- Node n = (Node) e;
- return "new org.apache.mrql.gen.Node(\""+n.name()+"\","+reify(n.children())+")";
- }
- }
-
- /** convert Tree constructions to code that construct these Trees (used in the Compiler) */
- public static String reify ( Trees ts ) {
- String s = "org.apache.mrql.gen.Trees.nil";
- for ( Tree c: ts )
- s += ".append("+reify(c)+")";
- return s;
- }
-
- /** return the list of free variables in e that are not in exclude list */
- public static Trees free_variables ( Tree e, Trees exclude ) {
- if (e == null)
- return #[];
- match e {
- case lambda(`x,`b):
- return free_variables(b,exclude.append(pattern_variables(x)));
- case let(`x,`u,`b):
- return free_variables(b,exclude.append(pattern_variables(x)))
- .append(free_variables(u,exclude));
- case Let(`x,`u,`b):
- return free_variables(b,exclude.append(pattern_variables(x)))
- .append(free_variables(u,exclude));
- case select(`u,from(...bs),`p):
- Trees ex = exclude;
- Trees fs = #[];
- for ( Tree b: bs )
- match b {
- case bind(`v,`x):
- fs = fs.append(free_variables(x,ex));
- ex = ex.append(pattern_variables(v));
- };
- return free_variables(p,ex).append(free_variables(u,ex)).append(fs);
- case `f(...as):
- Trees res = #[];
- for ( Tree a: as )
- res = res.append(free_variables(a,exclude));
- return res;
- case `v:
- if (v.is_variable() && v.toString().startsWith("x_") && !exclude.member(v))
- return #[`v];
- };
- return #[];
- }
-
- /** count the occurences of x in e */
- public static int occurences ( Tree x, Tree e ) {
- if (x.equals(e))
- return 1;
- match e {
- case `f(...as):
- int i = 0;
- for ( Tree a: as )
- i += occurences(x,a);
- return i;
- };
- return 0;
- }
-
- /** return true if x is equal to y modulo variable substitution */
- public static boolean alpha_equivalent ( Tree x, Tree y, SymbolTable st ) {
- match #<T(`x,`y)> {
- case T(lambda(`vx,`bx),lambda(`vy,`by)):
- if (!vx.equals(vy))
- st.insert(vx.toString(),vy);
- return alpha_equivalent(bx,by,st);
- case T(`f(...xs),`g(...ys)):
- if (!f.equals(g) || xs.length() != ys.length())
- return false;
- for ( ; !xs.is_empty(); xs = xs.tail(), ys = ys.tail() )
- if (!alpha_equivalent(xs.head(),ys.head(),st))
- return false;
- return true;
- case T(`v,`w):
- if (v.is_variable() && w.is_variable())
- return v.equals(w) || (st.lookup(v.toString()) != null
- && st.lookup(v.toString()).equals(w));
- };
- return x.equals(y);
- }
-
- private static SymbolTable alpha_symbol_table = new SymbolTable();
-
- /** return true if x is equal to y modulo variable substitution */
- public static boolean alpha_equivalent ( Tree x, Tree y ) {
- alpha_symbol_table.begin_scope();
- boolean b = alpha_equivalent(x,y,alpha_symbol_table);
- alpha_symbol_table.end_scope();
- return b;
- }
-
- /** translate a simplified select MRQL query to an algebraic form */
- public static Tree translate_select ( Tree e ) {
- match e {
- case select(`u,from(),where(true)):
- return #<bag(`(translate_select(u)))>;
- case select(`u,from(),where(`c)):
- return #<if(`(translate_select(c)),bag(`(translate_select(u))),bag())>;
- case select(`u,from(bind(`v,`d),...bl),where(`c)):
- Tree n = translate_select(#<select(`u,from(...bl),where(`c))>);
- return #<cmap(lambda(`v,`n),`(translate_select(d)))>;
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(translate_select(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- /** the MRQL top-level interfacse to evaluate a single MRQL expression or command */
- public static void top_level ( Tree expr ) {
- TopLevel.evaluate_top_level(expr);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Tuple.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Tuple.java b/src/main/java/core/Tuple.java
deleted file mode 100644
index bced3dd..0000000
--- a/src/main/java/core/Tuple.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for Tuples */
-final public class Tuple extends MRData {
- private final static long serialVersionUID = 723385754575L;
-
- MRData[] tuple;
-
- public Tuple ( int size ) {
- tuple = new MRData[size];
- }
-
- public Tuple ( final MRData ...as ) {
- tuple = as;
- }
-
- /** the number of elements in the tuple */
- public short size () { return (short)tuple.length; }
-
- public void materializeAll () {
- for (MRData e: tuple)
- e.materializeAll();
- };
-
- /** the i'th element of the tuple */
- public MRData get ( int i ) {
- return tuple[i];
- }
-
- /** the first element of the tuple */
- public MRData first () { return tuple[0]; }
-
- /** the second element of the tuple */
- public MRData second () { return tuple[1]; }
-
- /** replace the i'th element of a tuple with new data and return a new value */
- public MRData set ( int i, MRData data, MRData ret ) {
- tuple[i] = data;
- return ret;
- }
-
- /** replace the i'th element of a tuple with new data */
- public Tuple set ( int i, MRData data ) {
- tuple[i] = data;
- return this;
- }
-
- final public void write ( DataOutput out ) throws IOException {
- if (tuple.length == 0)
- out.writeByte(MRContainer.NULL);
- else if (tuple.length == 2) {
- out.writeByte(MRContainer.PAIR);
- tuple[0].write(out);
- tuple[1].write(out);
- } else if (tuple.length == 3) {
- out.writeByte(MRContainer.TRIPLE);
- tuple[0].write(out);
- tuple[1].write(out);
- tuple[2].write(out);
- } else {
- out.writeByte(MRContainer.TUPLE);
- WritableUtils.writeVInt(out,tuple.length);
- for (short i = 0; i < tuple.length; i++)
- tuple[i].write(out);
- }
- }
-
- final public static Tuple read ( DataInput in ) throws IOException {
- int n = WritableUtils.readVInt(in);
- Tuple t = new Tuple(n);
- for ( short i = 0; i < n; i++ )
- t.tuple[i] = MRContainer.read(in);
- return t;
- }
-
- final public static Tuple read2 ( DataInput in ) throws IOException {
- return new Tuple(MRContainer.read(in),MRContainer.read(in));
- }
-
- final public static Tuple read3 ( DataInput in ) throws IOException {
- return new Tuple(MRContainer.read(in),MRContainer.read(in),MRContainer.read(in));
- }
-
- public void readFields ( DataInput in ) throws IOException {
- int n = WritableUtils.readVInt(in);
- tuple = new Tuple[n];
- for ( short i = 0; i < n; i++ )
- tuple[i] = MRContainer.read(in);
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof Tuple);
- Tuple t = (Tuple) x;
- for ( short i = 0; i < tuple.length && i < t.tuple.length; i++ ) {
- int c = get(i).compareTo(t.get(i));
- if (c < 0)
- return -1;
- else if (c > 0)
- return 1;
- };
- if (tuple.length > t.tuple.length)
- return 1;
- else if (tuple.length < t.tuple.length)
- return -1;
- else return 0;
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- try {
- int n = WritableComparator.readVInt(x,xs);
- int s = WritableUtils.decodeVIntSize(x[xs]);
- for ( short i = 0; i < n; i++ ) {
- int k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
- if (k != 0)
- return k;
- s += size[0];
- };
- size[0] = s+1;
- return 0;
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- final public static int compare2 ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- int k = MRContainer.compare(x,xs,xl,y,ys,yl,size);
- if (k != 0)
- return k;
- int s = size[0];
- k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
- if (k != 0)
- return k;
- size[0] += s+1;
- return 0;
- }
-
- final public static int compare3 ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- int k = MRContainer.compare(x,xs,xl,y,ys,yl,size);
- if (k != 0)
- return k;
- int s = size[0];
- k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
- if (k != 0)
- return k;
- s += size[0];
- k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
- if (k != 0)
- return k;
- size[0] += s+1;
- return 0;
- }
-
- public boolean equals ( Object x ) {
- if (!(x instanceof Tuple))
- return false;
- Tuple xt = (Tuple) x;
- if (xt.tuple.length != tuple.length)
- return false;
- for ( short i = 0; i < tuple.length; i++ )
- if (!xt.get(i).equals(get(i)))
- return false;
- return true;
- }
-
- public int hashCode () {
- int h = 127;
- for ( short i = 1; i < tuple.length; i++ )
- h ^= get(i).hashCode();
- return Math.abs(h);
- }
-
- public String toString () {
- if (size() == 0)
- return "()";
- String s = "("+get((short)0);
- for ( short i = 1; i < tuple.length; i++ )
- s += ","+get(i);
- return s+")";
- }
-}
[04/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/TypeInference.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/TypeInference.gen b/src/main/java/core/TypeInference.gen
deleted file mode 100644
index 5276ae1..0000000
--- a/src/main/java/core/TypeInference.gen
+++ /dev/null
@@ -1,1356 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.*;
-
-
-/** the type inference/checker for MRQL expressions and algebraic forms */
-public class TypeInference extends Translator {
-
- private static Tree make_tuple ( Trees pl ) {
- if (pl.length() == 1)
- return pl.head();
- return #<tuple(...pl)>;
- }
-
- public static Tree make_persistent_type ( Tree tp ) {
- match tp {
- case `f(...al):
- Trees bs = #[];
- for ( Tree a: al )
- bs = bs.append(make_persistent_type(a));
- String g = persistent_collection(f);
- return #<`g(...bs)>;
- };
- return tp;
- }
-
- private static String max_collection ( String x, String y ) {
- boolean is_bag = x.equals("Bag") || y.equals("Bag") || x.equals("bag") || y.equals("bag");
- boolean is_persistent = x.equals("Bag") || y.equals("Bag") || x.equals("List") || y.equals("List");
- return (is_bag) ? ((is_persistent) ? "Bag" : "bag") : ((is_persistent) ? "List" : "list");
- }
-
- private static boolean numerical ( String tp ) {
- return #[short,int,long,float,double].member(#<`tp>);
- }
-
- static void type_error ( Tree e, String msg ) {
- System.err.println("*** Type error (line: "+e.line+", position: "+e.position+")");
- if (Config.trace && type_env.iterator().hasNext()) {
- msg += "\nVariable Types:";
- for ( String var: type_env )
- msg += "\n "+var + ": " + print_type(type_env.lookup(var));
- };
- System.err.println("*** "+msg);
- throw new Error("Type Error");
- }
-
- /** given that pattern has type tp, bind the pattern variables to types */
- private static Trees bind_pattern_type ( Tree pattern, Tree tp ) {
- Trees args = #[];
- match pattern {
- case tuple(...pl):
- match tp {
- case tuple(...tl):
- if (tl.length() != pl.length())
- type_error(pattern,"Tuple pattern "+print_query(pattern)+" must have exactly "
- +tl.length()+" components");
- int i = 0;
- for ( Tree p: pl )
- args = args.append(bind_pattern_type(p,tl.nth(i++)));
- case `etp: type_error(pattern,"Wrong pattern: found "+print_query(pattern)
- +" but expected a pattern that matches the type "+print_type(etp));
- };
- case record(...bl):
- Trees attrs = #[];
- match tp {
- case record(...tl):
- for ( Tree b: bl )
- match b {
- case bind(`n,`p):
- boolean found = false;
- if (attrs.member(n))
- type_error(pattern,"Duplicate record attribute name: "+n
- +" in pattern "+print_query(pattern));
- attrs = attrs.append(n);
- for ( Tree t: tl )
- match t {
- case bind(`nt,`pt):
- if (!nt.equals(n))
- fail;
- found = true;
- args = args.append(bind_pattern_type(p,pt));
- };
- if (!found)
- type_error(pattern,"Wrong record component: "+n
- +" in pattern "+print_query(pattern)
- +" (expected one from "+print_type(tp)+")");
- };
- case `etp: type_error(pattern,"Wrong pattern: found "+print_query(pattern)
- +" but expected a pattern that matches the type "+print_type(etp));
- };
- case typed(`p,`t):
- if (subtype(t,tp))
- args = bind_pattern_type(p,t);
- else type_error(pattern,"Type "+print_type(t)+" in pattern "+print_query(pattern)
- +" does not match the expected type "+print_type(tp));
- case list(...pl):
- match tp {
- case list(`etp):
- for ( Tree p: pl )
- args = args.append(bind_pattern_type(p,etp));
- case `stp: type_error(pattern,"List pattern "+print_query(pattern)
- +" can only be used for lists (found "+print_type(tp)+")");
- };
- case call(`c,...s):
- Tree ci = data_constructors.lookup(c.toString());
- if (ci == null)
- type_error(pattern,"Undefined data constructor "+c+" in pattern "+print_query(pattern));
- match ci {
- case `dname(`n,`dtp):
- if (!subtype(tp,expand(#<`dname>)))
- type_error(pattern,"Cannot use the data constructor "+print_query(pattern)
- +" in a pattern that expects type "+print_type(tp));
- args = args.append(bind_pattern_type(s.length() == 1 ? s.head() : #<tuple(...s)>,
- dtp));
- };
- case any: ;
- case `v:
- if (v.is_variable()) {
- args = args.append(v);
- type_env.insert(v.toString(),tp);
- } else if (!subtype(type_inference2(v),tp))
- type_error(pattern,"The constant "+v+" in pattern "
- +print_query(pattern)+" is not of type "+print_type(tp));
- };
- return args;
- }
-
- private static short needs_coerce ( Tree x, Tree y ) {
- if (x.is_variable() && numerical(x.toString())
- && y.is_variable() && numerical(y.toString())) {
- short cx = MRContainer.type_code(x.toString());
- short cy = MRContainer.type_code(y.toString());
- return (cx == cy) ? -1 : (cx > cy) ? cx : cy;
- };
- return -1;
- }
-
- /** type equality in MRQL is structured equality, not named equality */
- public static boolean equal_types ( Tree tx, Tree ty ) {
- tx = expand(tx);
- ty = expand(ty);
- if (tx.equals(ty))
- return true;
- match tx {
- case `f(...xs):
- match ty {
- case `g(...ys):
- if (f.equals(g) && xs.length() == ys.length()) {
- for ( ; !ys.is_empty(); xs = xs.tail(), ys = ys.tail() )
- if (!equal_types(xs.head(),ys.head()))
- return false;
- return true;
- }
- }
- };
- return false;
- }
-
- /** is the collection type name S a subtype of that of T?
- List \lt Bag \lt bag and List \lt list \lt bag
- */
- public static boolean subtype ( String S, String T ) {
- return S.equals(T)
- || (S.equals("List") && T.equals("list"))
- || (S.equals("List") && T.equals("Bag"))
- || (S.equals("list") && T.equals("bag"))
- || (S.equals("Bag") && T.equals("bag"));
- }
-
- /** is the type tx a subtype of type ty? */
- public static boolean subtype ( Tree tx, Tree ty ) {
- tx = expand(tx);
- ty = expand(ty);
- if (ty.equals(#<any>))
- return true;
- if (equal_types(tx,ty))
- return true;
- if (tx.is_variable() && numerical(tx.toString())
- && ty.is_variable() && numerical(ty.toString()))
- return MRContainer.type_code(tx.toString()) <= MRContainer.type_code(ty.toString());
- match tx {
- case `T(`tex):
- if (!is_collection(T))
- fail;
- match ty {
- case `S(`tey):
- if (is_collection(S))
- return subtype(T,S) && subtype(tex,tey);
- else fail
- };
- case union(...):
- if (ty.equals(#<union>)) // used for XML functions
- return true;
- else fail
- case `f(...xs):
- match ty {
- case `g(...ys):
- if (f.equals(g))
- return subtype(xs,ys);
- }
- };
- return false;
- }
-
- public static boolean subtype ( Trees ts1, Trees ts2 ) {
- if (ts1.length() != ts2.length())
- return false;
- for ( Trees s1 = ts1, s2 = ts2; !s1.is_empty(); s1=s1.tail(), s2=s2.tail() )
- if (!subtype(s1.head(),s2.head()))
- return false;
- return true;
- }
-
- public static int compare_types ( Tree t1, Tree t2 ) {
- if (t1.equals(#<any>) && t2.equals(#<any>))
- return 0;
- else if (t2.equals(#<any>))
- return -1;
- else if (t1.equals(#<any>))
- return 1;
- else if (t1 instanceof VariableLeaf && t2 instanceof VariableLeaf)
- return MRContainer.type_code(((VariableLeaf)t1).value())
- -MRContainer.type_code(((VariableLeaf)t2).value());
- else if (t1 instanceof VariableLeaf)
- return -1;
- else return 1;
- }
-
- /** if the expression at loc.head has type tx that is a subtype of ty, coerce it to ty
- * NOTE: it destructively changes the expression at loc
- */
- private static void coerce ( Tree tx, Tree ty, Trees loc ) {
- tx = expand(tx);
- ty = expand(ty);
- if (ty.equals(#<any>) || equal_types(tx,ty))
- return;
- if (tx.is_variable() && numerical(tx.toString())
- && ty.is_variable() && numerical(ty.toString())) {
- loc.head = #<call(coerce,`(loc.head), // destructive
- `(MRContainer.type_code(ty.toString())))>;
- return;
- };
- match tx {
- case `T(`tex):
- if (!is_collection(T))
- fail;
- match ty {
- case `S(`tey):
- if (!is_collection(S))
- fail;
- if (is_persistent_collection(T) && !is_persistent_collection(S))
- loc.head = #<Collect(`(loc.head))>; // destructive
- if (subtype(tex,tey) && unify(tex,tey) == null) {
- Tree nv = new_var();
- Tree b = #<bag(`nv)>;
- coerce(tex,tey,((Node)b).children);
- loc.head = #<cmap(lambda(`nv,`b),`(loc.head))>; // destructive
- return;
- }
- };
- case tuple(...xs):
- match ty {
- case tuple(...ys):
- Tree nv = new_var();
- Trees nt = #[];
- int i = 0;
- for ( Trees xl = xs, yl = ys; xl != #[] && yl != #[]; xl = xl.tail, yl = yl.tail ) {
- Trees dest = #[nth(`nv,`(i++))];
- coerce(xl.head,yl.head,dest);
- nt = nt.append(dest);
- };
- loc.head = #<let(`nv,`(loc.head),tuple(...nt))>; // destructive
- }
- case record(...xs):
- match ty {
- case record(...ys):
- Tree nv = new_var();
- Trees nt = #[];
- for ( Tree x: xs )
- match x {
- case bind(`ax,`tex):
- for ( Tree y: ys )
- match y {
- case bind(`ay,`tey):
- if (equal_types(ax,ay)) {
- Tree b = #<bind(`ax,project(`nv,`ax))>;
- nt = nt.append(b);
- coerce(tex,tey,((Node)b).children.tail);
- }
- }
- };
- loc.head = #<let(`nv,`(loc.head),record(...nt))>; // destructive
- }
- }
- }
-
- /** pure-mans type-unification (the any type unifies with everything) */
- private static Trees unify ( Trees ts1, Trees ts2 ) {
- Trees s = #[];
- if (ts1.length() != ts2.length())
- return null;
- for ( Trees s1 = ts1, s2 = ts2; !s1.is_empty(); s1=s1.tail(), s2=s2.tail() ) {
- Tree t = unify(s1.head(),s2.head());
- if (t == null)
- return null;
- else s = s.append(t);
- };
- return s;
- }
-
- /** pure-mans type-unification (the any type unifies with everything) */
- public static Tree unify ( Tree t1, Tree t2 ) {
- t1 = expand(t1);
- t2 = expand(t2);
- if (t1.equals(#<any>))
- return t2;
- if (t2.equals(#<any>) || equal_types(t1,t2))
- return t1;
- match t1 {
- case `T(`t):
- match t2 {
- case `S(`s):
- if (!T.equals(S))
- fail;
- if (!is_collection(T))
- fail;
- Tree ts = unify(t,s);
- if (ts != null)
- return #<`T(`ts)>;
- }
- case `f(...ts1):
- match t2 {
- case `g(...ts2):
- Trees s = unify(ts1,ts2);
- if (f.equals(g) && s != null)
- return #<`f(...s)>;
- }
- };
- return null;
- }
-
- /** if the types tx and ty do not unify, try to coerce them */
- static Tree unify ( Tree tx, Tree ty, Trees destx, Trees desty ) {
- Tree tp = unify(tx,ty);
- if (tp != null)
- return tp;
- else if (subtype(tx,ty)) {
- coerce(tx,ty,destx);
- return ty;
- } else if (subtype(ty,tx)) {
- coerce(ty,tx,desty);
- return tx;
- } else return null;
- }
-
- /** find a type from the list of types ts that is the supertype of all other types */
- private static Tree maximum_type ( Trees ts ) {
- Tree maxt = ts.head;
- for ( Tree t: ts.tail )
- if (subtype(maxt,t) || maxt.equals(#<any>))
- maxt = t;
- return maxt;
- }
-
- /** if the type tp is a named type, expand it using its definition */
- public static Tree expand ( Tree tp ) {
- if (!tp.is_variable())
- return tp;
- Tree rt = global_datatype_env.lookup(tp.toString());
- if (rt != null)
- return expand(rt);
- rt = type_names.lookup(tp.toString());
- if (rt == null)
- return tp;
- else return expand(rt);
- }
-
- /** infer the type of an expression and expand it if necessary
- * @param e the expression
- * @return the type of e
- */
- public static Tree type_inference2 ( Tree e ) {
- return expand(type_inference(e));
- }
-
- /** infer the type of an expression
- * @param e the expression
- * @return the type of e
- */
- public static Tree type_inference ( Tree e ) {
- match e {
- case select(`opt_dist,`u,from(...bl),where(`c),groupby(...gs),orderby(...os)):
- type_env.begin_scope();
- Trees dvs = #[];
- String max_type = "list";
- for ( Tree b: bl )
- match b {
- case bind(`p,`d):
- match type_inference2(d) {
- case `T(`tp):
- if (!is_collection(T))
- fail;
- dvs = dvs.append(bind_pattern_type(p,tp));
- max_type = max_collection(T,max_type);
- case `ftp:
- type_error(e,"The from-binding domain "+print_query(d)+" in "
- +print_query(e)+" must be a collection (found "
- +print_type(ftp)+")");
- }
- };
- Tree ctp = type_inference2(c);
- if (unify(ctp,#<bool>) == null)
- type_error(e,"The predicate "+print_query(c)+" in "+print_query(e)
- +" must be a boolean (found "+print_type(ctp)+")");
- match #<groupby(...gs)> {
- case groupby(`h,...gl):
- Trees pvs = #[];
- for ( Tree g: gl )
- match g {
- case bind(`gp,`gd):
- bind_pattern_type(gp,type_inference2(gd));
- pvs = pvs.append(pattern_variables(gp));
- };
- // find the type of the partition variable
- Trees partl = #[];
- for ( Tree dv: pattern_variables(#<tuple(...dvs)>) )
- partl = partl.append(#<bind(`dv,`(type_env.lookup(dv.toString())))>);
- type_env.insert("partition",#<bag(record(...partl))>);
- // lift domain variables to bags
- for ( Tree dv: dvs )
- if (!pvs.member(dv))
- type_env.insert(dv.toString(),
- #<bag(`(type_env.lookup(dv.toString())))>);
- Tree htp = type_inference2(h);
- if (unify(htp,#<bool>) == null)
- type_error(e,"The group-by predicate "+print_query(h)+" in "+print_query(e)
- +" must be a boolean (found "+print_type(htp)+")");
- };
- match #<orderby(...os)> {
- case orderby(`l,...ol):
- if (!l.equals(#<none>)) {
- Tree ltp = type_inference2(l);
- if (unify(ltp,#<int>) == null)
- type_error(e,"The limit "+print_query(l)+" in "+print_query(e)
- +" must be an int (found "+print_type(ltp)+")");
- };
- for ( Tree o: ol)
- type_inference2(o);
- Tree rtp = type_inference2(u);
- type_env.end_scope();
- return (is_persistent_collection(max_type)) ? #<List(`rtp)> : #<list(`rtp)>;
- };
- Tree rtp = type_inference2(u);
- type_env.end_scope();
- return (is_persistent_collection(max_type)) ? #<Bag(`rtp)> : #<bag(`rtp)>;
- case select(`u,from(...bl),where(`c)):
- String max_type = "list";
- for ( Tree b: bl )
- match b {
- case bind(`v,`d):
- match type_inference2(d) {
- case `T(`tp):
- if (!is_collection(T))
- fail;
- type_env.insert(v.toString(),tp);
- max_type = max_collection(T,max_type);
- case _: type_error(e,"Expected a collection: "+print_query(d)
- +" in "+print_query(e));
- }
- };
- if (unify(type_inference2(c),#<bool>) != null) {
- Tree tp = type_inference(u);
- return (is_persistent_collection(max_type)) ? #<Bag(`tp)> : #<bag(`tp)>;
- } else type_error(e,"The select predicate must be boolean: "+print_query(e));
- case let_bind(`p,`u,`b):
- type_env.begin_scope();
- bind_pattern_type(p,type_inference2(u));
- Tree tp = type_inference2(b);
- type_env.end_scope();
- return tp;
- case let(`p,`u,`b):
- bind_pattern_type(p,type_inference2(u));
- return type_inference2(b);
- case case(`u,...cs):
- Tree tp = type_inference2(u);
- Trees ts = #[];
- for ( Tree c: cs )
- match c {
- case case(`p,`b):
- type_env.begin_scope();
- bind_pattern_type(p,tp);
- ts = ts.append(type_inference2(b));
- type_env.end_scope();
- };
- Tree maxt = maximum_type(ts);
- for ( ; cs != #[] && ts != #[]; cs = cs.tail, ts = ts.tail )
- match cs.head {
- case case(`p,`b):
- if (subtype(ts.head,maxt))
- coerce(ts.head,maxt,((Node)(cs.head)).children.tail);
- else type_error(cs.head,"Mismatched case output: "+b);
- };
- return maxt;
- case lambda(`v,`b):
- if(!v.is_variable())
- fail;
- if (type_env.lookup(v.toString()) == null)
- type_env.insert(v.toString(),#<any>);
- return #<arrow(`(type_env.lookup(v.toString())),`(type_inference(b)))>;
- case function(tuple(...params),`outp,`body):
- Trees as = #[];
- for ( Tree param: params )
- match param {
- case `bind(`v,`tp):
- type_env.insert(v.toString(),tp);
- as = as.append(tp);
- };
- Tree btp = type_inference(body);
- if (!subtype(btp,outp))
- type_error(e,"The type of the function body "+print_type(btp)
- +" does not match the expected type "+print_type(outp)+"\n"+e);
- if (unify(btp,outp) == null) { // the body needs coercion
- Trees b = #[`body];
- coerce(btp,outp,b);
- body = b.head;
- };
- return #<arrow(tuple(...as),`outp)>;
- case call(source,binary,`f,`tp):
- return tp;
- case call(source,binary,`f):
- if (!f.is_string())
- type_error(e,"The source file must be a constant string: "+print_query(f));
- Tree tp = null;
- if (Config.hadoop_mode)
- tp = Plan.get_type(f.stringValue());
- else tp = MapReduceAlgebra.get_type(f.stringValue());
- if (tp == null)
- type_error(e,"Cannot find the type of file "+f);
- ((Node)e).children = ((Node)e).children.append(tp); // destructive
- return tp;
- case call(source,`parser,`f,...args):
- if (!parser.is_variable())
- type_error(e,"The parser must be a constant name: "+print_query(parser));
- if (unify(type_inference(f),#<string>) == null)
- type_error(e,"The source file must be a string: "+print_query(f));
- try {
- Class<? extends Parser> pc = DataSource.parserDirectory.get(parser.toString());
- if (pc == null)
- type_error(e,"Unrecognized parser: "+parser);
- Parser p = pc.newInstance();
- p.initialize(args);
- return #<Bag(`(p.type()))>;
- } catch (Exception x) {
- type_error(e,"Unrecognized parser type: "+parser);
- }
- case typed(null,`tp):
- return tp;
- case typed(`f(...),`tp):
- if (f.equals("tagged_union") || f.equals("union_value"))
- return tp;
- else fail
- case typed(`x,`tp):
- if (tp.is_variable() && !tp.equals(#<string>)
- && MRContainer.type_code(tp.toString()) >= 0) {
- Tree tx = type_inference(x);
- if (tx.is_variable() && !tx.equals(#<string>)
- && MRContainer.type_code(tx.toString()) >= 0)
- return tp;
- else type_error(e,"Expression "+print_query(x)+" of type "+print_type(tx)
- +" cannot be coerced to type "+tp);
- };
- Tree tx = type_inference(x);
- if (!subtype(tx,tp))
- type_error(e,"Expression "+print_query(x)+" of type "+print_type(tx)
- +" cannot be coerced to type "+print_type(tp));
- if (unify(tx,tp) == null) // need to coerce x
- coerce(tx,tp,((Node)e).children);
- return tp;
- case tuple(...el):
- Trees s = #[];
- for ( Tree a: el )
- s = s.append(type_inference(a));
- return #<tuple(...s)>;
- case call(coerce,`x,`n):
- return #<`(MRContainer.type_names[(int)n.longValue()])>;
- case record(...el):
- Trees s = #[];
- Trees attrs = #[];
- for ( Tree a: el )
- match a {
- case bind(`v,`b):
- s = s.append(#<bind(`v,`(type_inference(b)))>);
- if (attrs.member(v))
- type_error(e,"Duplicate record attribute name: "+v+" in "+print_query(e));
- attrs = attrs.append(v);
- };
- return #<record(...s)>;
- case union_tag(`x):
- return #<int>;
- case `T(...el):
- if (!is_collection(T))
- fail;
- if (el.is_empty())
- return #<`T(any)>;
- Trees ts = #[];
- for ( Tree t: el )
- ts = ts.append(type_inference(t));
- Tree maxt = maximum_type(ts);
- for ( ; el != #[] && ts != #[]; el = el.tail, ts = ts.tail )
- if (subtype(ts.head,maxt))
- coerce(ts.head,maxt,el);
- else type_error(e,"Incompatible values in "+T+" construction: "+print_query(e));
- return #<`T(`maxt)>;
- case nth(`a,`n):
- if (!n.is_long())
- type_error(e,"Tuple index must be an integer: "+print_query(e));
- int i = (int)n.longValue();
- match type_inference2(a) {
- case tuple(...ts):
- if (i < 0 || i >= ts.length())
- type_error(e,"Tuple index outside bounds: "+print_query(e));
- return ts.nth(i);
- case `S(tuple(...ts)):
- if (!is_collection(S))
- fail;
- if (i < 0 || i >= ts.length())
- type_error(e,"Tuple index outside bounds: "+print_query(e));
- return #<`S(`(ts.nth(i)))>;
- case `tp:
- type_error(e,"Tuple index must be applied to a tuple: "
- +print_query(a)+" of type: "+print_type(tp)+" in "+print_query(e));
- };
- case project(`a,`v): // highly overloaded
- Tree ta = type_inference(a);
- match ta {
- case XML:
- return #<list(XML)>;
- case `S(`tp):
- if (is_collection(S) && (tp.equals(#<XML>) || tp.equals(TopLevel.xml_type)))
- return #<`S(XML)>;
- };
- if (ta.equals(TopLevel.xml_type))
- return #<list(XML)>;
- match expand(ta) {
- case record(...ts):
- for ( Tree t: ts )
- match t {
- case bind(`w,`tp):
- if (equal_types(w,v))
- return tp;
- };
- type_error(e,"Record "+print_query(a)+" does not have a component "+v);
- case union(...tl):
- Trees s = #[];
- for ( Tree t: tl )
- match t {
- case `c(record(...ts)):
- for ( Tree tn: ts )
- match tn {
- case bind(`w,`tp):
- if (equal_types(w,v))
- s = s.append(tp);
- };
- case `c(bag(tuple(string,`tv))):
- s = s.append(tv);
- };
- if (s.length() == 0)
- type_error(e,"Wrong record projection "+print_query(e)
- + " over the union "+print_type(ta));
- else if (s.length() > 1)
- type_error(e,"Ambiguous record projection "+print_query(e)
- + " over the union "+print_type(ta));
- return s.head();
- case `S(`ttp):
- if (!is_collection(S))
- fail;
- match expand(ttp) {
- case record(...ts):
- for ( Tree t: ts )
- match t {
- case bind(`w,`tp):
- if (equal_types(w,v))
- return #<`S(`tp)>;
- };
- type_error(e,"The record collection "+print_query(a)
- +" does not have a component "+v);
- case tuple(string,`tv):
- return tv;
- case union(...tl):
- Trees s = #[];
- for ( Tree t: tl )
- match t {
- case `c(record(...ts)):
- for ( Tree tn: ts )
- match tn {
- case bind(`w,`tp):
- if (equal_types(w,v))
- s = s.append(tp);
- };
- case `c(bag(tuple(string,`tv))):
- s = s.append(tv);
- };
- if (s.length() == 0)
- type_error(e,"Wrong record projection "+print_query(e)
- + " over the union "+print_type(ta));
- else if (s.length() > 1)
- type_error(e,"Ambiguous record projection "+print_query(e)
- + " over the union "+print_type(ta));
- return #<`S(...s)>;
- };
- case `t:
- type_error(e,"The record projection "+print_query(e)
- + " cannot apply to the type "+print_type(t));
- };
- case index(`a,`i):
- Tree ti = type_inference2(i);
- match type_inference2(a) {
- case bag(tuple(`tk,`tv)):
- if (subtype(ti,tk)) {
- coerce(ti,tk,((Node)e).children.tail);
- return tv;
- } else fail
- case Bag(tuple(`tk,`tv)):
- if (subtype(ti,tk)) {
- coerce(ti,tk,((Node)e).children.tail);
- return tv;
- } else fail
- case list(`tp):
- if (unify(ti,#<int>) != null)
- return tp;
- else type_error(e,"List index must be an integer: "+print_query(e));
- case List(`tp):
- if (unify(ti,#<int>) != null)
- return tp;
- else type_error(e,"List index must be an integer: "+print_query(e));
- case union(...tl):
- Trees s = #[];
- for ( Tree t: tl )
- match expand(t) {
- case `c(bag(tuple(`tk,`tv))):
- if (unify(ti,tk) != null)
- s = s.append(tv);
- else fail
- case `c(list(`tp)):
- if (unify(ti,#<int>) != null)
- s = s.append(tp);
- else fail
- };
- if (s.length() == 0)
- type_error(e,"Wrong indexing "+print_query(e)
- + " in union "+print_type(#<union(...tl)>));
- else if (s.length() > 1)
- type_error(e,"Ambiguous indexing "+print_query(e)
- + " in union "+print_type(#<union(...tl)>));
- return s.head();
- case `tp: type_error(e,"Indexing is not allowed for type "+print_type(tp)
- +" with index "+print_type(ti)+" in "+print_query(e));
- };
- case range(`u,`i,`j):
- if (unify(type_inference2(i),#<int>) == null
- || unify(type_inference2(j),#<int>) == null)
- type_error(e,"Range indexes must be integer expressions: "+print_query(e));
- match type_inference2(u) {
- case list(`a): return #<list(`a)>;
- case List(`a): return #<list(`a)>;
- };
- type_error(e,"Range indexing must be applied to a list: "+print_query(e));
- case range(`min,`max):
- Tree tmin = type_inference(min);
- Tree tmax = type_inference(max);
- if (!subtype(tmin,#<long>))
- type_error(e,"Expected a long integer for min: "+print_query(min));
- else if (unify(tmin,#<long>) == null) // coerce
- coerce(tmin,#<long>,((Node)e).children);
- if (!subtype(tmax,#<long>))
- type_error(e,"Expected a long integer for max: "+print_query(max));
- else if (unify(tmax,#<long>) == null) // coerce
- coerce(tmax,#<long>,((Node)e).children.tail);
- return #<list(long)>;
- case call(gen,`min,`max,`size):
- return type_inference(#<gen(`min,`max,`size)>);
- case gen(`min,`max,`size):
- Tree tmin = type_inference(min);
- Tree tmax = type_inference(max);
- Tree tsize = type_inference(size);
- if (!subtype(tmin,#<long>))
- type_error(e,"Expected a long integer for min: "+print_query(min));
- else if (unify(tmin,#<long>) == null) // coerce
- coerce(tmin,#<long>,((Node)e).children);
- if (!subtype(tmax,#<long>))
- type_error(e,"Expected a long integer for max: "+print_query(max));
- else if (unify(tmax,#<long>) == null) // coerce
- coerce(tmax,#<long>,((Node)e).children.tail);
- if (!subtype(tsize,#<long>))
- type_error(e,"Expected a long integer for size: "+print_query(size));
- else if (unify(tsize,#<long>) == null) // coerce
- coerce(tsize,#<long>,((Node)e).children.tail.tail);
- return #<Bag(long)>;
- case dataset_size(`x):
- return #<long>;
- case groupBy(`u):
- Tree tp = type_inference2(u);
- match tp {
- case `T(tuple(`k,`v)):
- if (is_collection(T))
- return (is_persistent_collection(T))
- ? #<Bag(tuple(`k,list(`v)))>
- : #<bag(tuple(`k,list(`v)))>;
- };
- type_error(e,"Wrong groupBy: "+print_query(e)+" of type "+print_type(tp));
- case orderBy(`u):
- Tree tp = type_inference2(u);
- match tp {
- case `T(tuple(`k,`v)):
- if (is_collection(T))
- return (is_persistent_collection(T))
- ? #<List(tuple(`k,list(`v)))>
- : #<list(tuple(`k,list(`v)))>;
- };
- type_error(e,"Wrong orderBy: "+print_query(e)+" of type "+print_type(tp));
- case cmap(lambda(`v,`body),`s):
- match type_inference2(s) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- type_env.insert(v.toString(),a);
- match type_inference2(body) {
- case `S(`tp):
- if (!is_collection(S))
- fail;
- return #<`T(`tp)>;
- case _: type_error(e,"cmap must return a collection: "+print_query(e));
- };
- case `t: type_error(e,"cmap must be over a collection: "+print_query(e)
- +" (found type "+print_type(t)+")");
- };
- type_error(e,"Wrong cmap: "+print_query(e));
- case fold(lambda(`v,`body),`n,`s):
- match type_inference2(s) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- Tree tp = type_inference(n);
- type_env.insert(v.toString(),#<tuple(`a,tp)>);
- if (unify(type_inference2(body),tp) == null)
- type_error(e,"Wrong types in fold: "+print_query(e));
- return tp;
- };
- case join(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`vr,`br),`x,`y):
- match type_inference2(x) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- match type_inference2(y) {
- case `S(`b):
- if (!is_collection(S))
- fail;
- type_env.insert(v1.toString(),a);
- type_env.insert(v2.toString(),b);
- T = transient_collection(T);
- S = transient_collection(S);
- type_env.insert(vr.toString(),#<tuple(`T(`a),`S(`b))>);
- if (unify(type_inference2(b1),type_inference2(b2)) == null)
- type_error(e,"Incompatible keys in join ("+type_inference2(b1)
- +" and "+type_inference2(b2)+"): "+print_query(e));
- match type_inference(br) {
- case `S3(`ntp):
- if (!is_collection(S3))
- fail;
- S3 = persistent_collection(S3);
- return #<`S3(`ntp)>;
- };
- type_error(e,"The join reducer must return a collection: "+print_query(br));
- case _: type_error(e,"The right join input is not a collection: "+print_query(y));
- };
- case _: type_error(e,"The left join input is not a collection: "+print_query(x));
- };
- case crossProduct(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`vr,`br),`x,`y):
- match type_inference2(x) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- match type_inference2(y) {
- case `S(`b):
- if (!is_collection(S))
- fail;
- type_env.insert(v1.toString(),a);
- type_env.insert(v2.toString(),b);
- match type_inference2(b1) {
- case `S1(`w1):
- if (!is_collection(S1))
- fail;
- match type_inference2(b2) {
- case `S2(`w2):
- if (!is_collection(S2))
- fail;
- type_env.insert(vr.toString(),#<tuple(`w1,`w2)>);
- match type_inference(br) {
- case `S3(`ntp):
- if (!is_collection(S3))
- fail;
- S3 = persistent_collection(S3);
- return #<`S3(`ntp)>;
- };
- type_error(e,"The cross product reducer must return a collection: "+print_query(br));
- case _: type_error(e,"Wrong right mapper in a cross product: "+print_query(b2));
- };
- case _: type_error(e,"Wrong left mapper in a cross product: "+print_query(b1));
- };
- case _: type_error(e,"The right cross product input is not a collection: "+print_query(y));
- };
- case _: type_error(e,"The left cross product input is not a collection: "+print_query(x));
- };
- case mapReduce(lambda(`vm,`bm),lambda(`vr,`br),`X,_):
- match type_inference2(X) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- type_env.insert(vm.toString(),a);
- match type_inference2(bm) {
- case `S(tuple(`k,`b)):
- if (!is_collection(S))
- fail;
- type_env.insert(vr.toString(),#<tuple(`k,list(`b))>);
- match type_inference(br) {
- case `S3(`ntp):
- if (!is_collection(S3))
- fail;
- if (is_persistent_collection(T))
- S3 = persistent_collection(S3);
- return #<`S3(`ntp)>;
- };
- type_error(e,"The MapReduce reducer must return a collection: "+print_query(br));
- case _: type_error(e,"Wrong mapper in mapReduce: "+print_query(bm));
- }
- };
- type_error(e,"The MapReduce input is not a collection: "+print_query(X));
- case mapReduce2(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`vr,`br),`X,`Y,_):
- match type_inference2(X) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- match type_inference2(Y) {
- case `S(`b):
- if (!is_collection(S))
- fail;
- type_env.insert(v1.toString(),a);
- type_env.insert(v2.toString(),b);
- match type_inference2(b1) {
- case `S1(tuple(`k1,`w1)):
- if (!is_collection(S1))
- fail;
- match type_inference2(b2) {
- case `S2(tuple(`k2,`w2)):
- if (!is_collection(S2))
- fail;
- if (unify(k1,k2) == null)
- type_error(e,"incompatible keys in mapReduce2: "+print_query(e));
- S1 = transient_collection(S1);
- S2 = transient_collection(S2);
- type_env.insert(vr.toString(),#<tuple(`S1(`w1),`S2(`w2))>);
- match type_inference(br) {
- case `S3(`ntp):
- if (!is_collection(S3))
- fail;
- if (is_persistent_collection(T) && is_persistent_collection(S))
- S3 = persistent_collection(S3);
- return #<`S3(`ntp)>;
- };
- type_error(e,"The MapReduce2 reducer must return a collection: "+print_query(br));
- case _: type_error(e,"Wrong right mapper in mapReduce2: "+print_query(b2));
- };
- case _: type_error(e,"Wrong left mapper in mapReduce2: "+print_query(b1));
- };
- case _: type_error(e,"The right MapReduce2 input is not a collection: "+print_query(Y));
- };
- case _: type_error(e,"The left MapReduce2 input is not a collection: "+print_query(X));
- };
- case Collect(`x):
- match type_inference2(x) {
- case Bag(`tp): return #<bag(`tp)>;
- case List(`tp): return #<list(`tp)>;
- };
- type_error(e,"You can only cache persistent collections: "+e);
- case aggregate(lambda(`v,`b),`z,`X):
- match type_inference2(X) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- Tree ztp = type_inference2(z);
- type_env.insert(v.toString(),#<tuple(`ztp,`a)>);
- if (!subtype(ztp,type_inference2(b)))
- type_error(e,"Wrong accumulator: "+print_query(e));
- return ztp;
- };
- type_error(e,"Aggregation input is not a collection: "+print_query(e));
- case repeat(lambda(`p,`b),`s,...ns):
- if (!ns.is_empty() && unify(type_inference2(ns.head()),#<int>) == null)
- type_error(e,"The maximum number of steps in repeat must be an integer: "
- +print_query(ns.head()));
- Tree tp = type_inference2(s);
- bind_pattern_type(p,tp);
- match tp {
- case `T(`a):
- if (!is_collection(T))
- fail;
- match type_inference2(b) {
- case `S(`c): // transitive closure
- if (!is_collection(S))
- fail;
- if (unify(a,c) == null)
- fail;
- ((Node)e).name = #<closure>.toString(); // destructive
- return #<`T(`a)>;
- case `S(tuple(`w,`c)):
- if (!is_collection(S))
- fail;
- if (unify(c,#<bool>) == null)
- fail;
- if (unify(a,w) == null)
- fail;
- return #<`T(`a)>;
- case `t: type_error(e,"The repeat body must return a collection of type "
- +print_type(tp)+" or "+print_type(#<`T(tuple(`a,bool))>)
- +" (Found type: "+print_type(t)+")");
- }
- };
- type_error(e,"The repeat source must return a bag: "+print_query(e));
- case closure(lambda(`v,`b),`s,...ns):
- if (!ns.is_empty() && unify(type_inference2(ns.head()),#<int>) == null)
- type_error(e,"The maximum number of steps in closure must be an integer: "
- +print_query(ns.head()));
- match type_inference2(s) {
- case `T(`a):
- if (!is_collection(T))
- fail;
- type_env.insert(v.toString(),#<`T(`a)>);
- match type_inference2(b) {
- case `S(`tp):
- if (!is_collection(S))
- fail;
- if (unify(a,tp) == null)
- fail;
- return #<`T(`a)>;
- case `tp: type_error(e,"The closure body must return a collection of type "
- +print_type(#<`T(`a)>)+" or "+print_type(#<`T(tuple(`a,bool))>)
- +" (Found type: "+print_type(tp)+")");
- }
- };
- type_error(e,"The closure source must return a bag: "+print_query(e));
- case loop(lambda(`p,`b),`s,`n):
- if (unify(type_inference2(n),#<int>) == null)
- type_error(e,"The number of steps in loop must be an integer: "+print_query(n));
- Tree tp = type_inference2(s);
- bind_pattern_type(p,tp);
- Tree btp = type_inference2(b);
- if (unify(tp,btp) == null)
- type_error(e,"The type of the repeat body ("+print_type(btp)
- + ") and the type of the initial value ("+print_type(tp)+") do not match");
- return tp;
- case step(`x): // used in QueryPlan for repeat
- match type_inference(x) {
- case `T(`tp): return #<`T(tuple(`tp,bool))>;
- };
- case cstep(`x): // used in QueryPlan for closure
- return type_inference(x);
- case if(`p,`x,`y):
- if (unify(type_inference2(p),#<bool>) == null)
- type_error(e,"Expected a boolean predicate in if-then-else: "+print_query(p));
- Tree tx = type_inference2(x);
- Tree ty = type_inference2(y);
- Tree rt = unify(tx,ty,((Node)e).children.tail,((Node)e).children.tail.tail);
- if (rt == null)
- type_error(e,"Incompatible types in if-then-else: "+print_query(e));
- return rt;
- case call(inv,`x):
- return type_inference(x);
- case call(plus,`x,`y):
- Tree tx = type_inference2(x);
- Tree ty = type_inference2(y);
- match tx {
- case `T(`xt):
- if (!is_collection(T))
- fail;
- match ty {
- case `S(`yt):
- if (!is_collection(S))
- fail;
- Tree rt = unify(tx,ty,((Node)e).children.tail,((Node)e).children.tail.tail);
- if (rt == null)
- type_error(e,"Incompatible types in union/append: "+print_type(tx)+" and "+print_type(ty));
- return rt;
- }
- };
- fail
- case `f(`x,`y):
- if (! #[union,intersect,except].member(#<`f>))
- fail;
- Tree tx = type_inference2(x);
- Tree ty = type_inference2(y);
- match tx {
- case `T(`t1):
- if (!is_collection(T))
- fail;
- Tree t = unify(tx,ty,((Node)e).children,((Node)e).children.tail);
- if (t != null)
- return t;
- };
- type_error(e,"Incompatible types in "+f+": "+print_type(tx)+" and "+print_type(ty));
- case call(member,`x,`y):
- Tree tx = type_inference2(x);
- Tree ty = type_inference2(y);
- match ty {
- case `T(`t1):
- if (!is_collection(T))
- fail;
- if (!subtype(tx,t1))
- type_error(e,"Incompatible types in member: "+print_type(tx)+" and "+print_type(ty));
- coerce(tx,t1,((Node)e).children.tail);
- return #<bool>;
- };
- case call(`f,`x,`y):
- if (! #[eq,neq,lt,leq,gt,geq].member(f))
- fail;
- Tree tx = type_inference2(x);
- Tree ty = type_inference2(y);
- if (!subtype(tx,ty) && !subtype(ty,tx))
- type_error(e,"Incompatible types in comparison "+f+": "
- +print_type(tx)+" and "+print_type(ty));
- if (unify(tx,ty) != null)
- return #<bool>;
- if (subtype(tx,ty))
- coerce(tx,ty,((Node)e).children.tail);
- else if (subtype(ty,tx))
- coerce(ty,tx,((Node)e).children.tail.tail);
- return #<bool>;
- case call(`f,`s):
- for ( Tree monoid: monoids )
- match monoid {
- case `aggr(`mtp,`plus,`zero,`unit):
- if (!aggr.equals(f.toString()))
- continue;
- match type_inference2(s) {
- case `T(`tp):
- if (!is_collection(T))
- type_error(e,"Aggregation must be over collections: "+s);
- if (unify(mtp,tp) == null)
- continue;
- Tree nv = new_var();
- type_env.begin_scope();
- type_env.insert(nv.toString(),tp);
- Tree t = type_inference2(#<apply(`unit,`nv)>);
- Tree tz = type_inference2(zero);
- type_env.end_scope();
- if (unify(t,tz) != null)
- return t;
- }
- };
- fail
- case call(avg,`s):
- return type_inference(#<call(div,typed(call(sum,`s),double),call(count,`s))>);
- case apply(lambda(`v,`b),`arg):
- type_env.begin_scope();
- type_env.insert(v.toString(),type_inference(arg));
- Tree tp = type_inference(b);
- type_env.end_scope();
- return tp;
- case call(`f,...al):
- Tree macro = global_macros.lookup(f.toString());
- if (macro == null)
- fail;
- match macro {
- case macro(params(...pl),`body):
- Tree b = body;
- if (pl.length() != al.length())
- fail;
- for ( ; !pl.is_empty(); pl = pl.tail(), al = al.tail() )
- b = subst(pl.head(),al.head(),b);
- return type_inference2(b);
- };
- fail
- case call(`f,...el):
- if (!f.is_variable() || global_vars.lookup(f.toString()) != null)
- fail;
- Tree ret = data_constructors.lookup(f.toString());
- if (ret != null)
- match ret {
- case `v(`n,`tp):
- if (unify(type_inference(make_tuple(el)),tp) != null)
- return #<`v>;
- else type_error(e,"Wrong data construction: "+print_query(e));
- };
- ret = type_env.lookup(f.toString());
- if (ret == null)
- ret = global_type_env.lookup(f.toString());
- if (ret != null)
- match ret {
- case arrow(`s,`d):
- Tree tp = type_inference(#<tuple(...el)>);
- if (!subtype(tp,s))
- type_error(e,"The domain of the anonymous function "+print_type(s)+" in "+print_query(e)
- +" does not match the argument types "+print_type(tp));
- if (unify(tp,s) == null) // coerce args
- match #<tuple(`tp,`s)> {
- case tuple(tuple(...txs),tuple(...tys)):
- for ( ; txs != #[]; txs = txs.tail, tys = tys.tail, el = el.tail )
- coerce(txs.head,tys.head,el);
- }
- return d;
- case _: type_error(e,"Expected a functional type for "+f+" (found "+print_type(ret)+")");
- };
- Trees tps = #[];
- for ( Tree a: el )
- tps = tps.append(type_inference(a));
- for ( Tree fnc: functions )
- match fnc {
- case `fn(`otp,...tl):
- if (!fn.equals(f.toString()) || !subtype(tps,tl))
- fail;
- if (f.equals(#<XMLchildren>))
- return #<list(XML)>;
- else if (f.equals(#<XMLattributes>))
- return #<list(string)>;
- else if (f.equals(#<XMLattribute>) && collection_type(otp))
- return #<list(string)>;
- else return otp;
- };
- type_error(e,"Undefined function "+f+" over arguments of type "+print_type(#<tuple(...tps)>));
- case apply(`f,`u):
- match type_inference2(f) {
- case arrow(`s,`d):
- Tree tp = type_inference(u);
- if (!subtype(tp,s))
- type_error(e,"The domain of the anonymous function "+print_type(s)+" in "+print_query(e)
- +" does not match the argument types "+print_type(tp));
- if (unify(tp,s) == null) // coerce args
- coerce(tp,s,((Node)e).children.tail);
- return d;
- case `tp: type_error(e,"Expected a functional type for "+f+" (found "+print_type(tp)+")");
- };
- case callM(`f,_,...el):
- return type_inference(#<call(`f,...el)>);
- case true: return #<bool>;
- case false: return #<bool>;
- case null: return #<any>;
- case `v:
- if (!v.is_variable())
- fail;
- Tree ret1 = type_env.lookup(v.toString());
- Tree ret2 = global_type_env.lookup(v.toString());
- if (ret1 == null)
- if (ret2 == null) {
- Tree ret = global_vars.lookup(v.toString());
- if (ret == null) {
- String msg = "";
- if (!Config.trace && type_env.iterator().hasNext()) {
- msg += "\nVariable Types:";
- for ( String var: type_env )
- msg += "\n "+var + ": " + print_type(type_env.lookup(var));
- };
- type_error(e,"Undefined variable: "+v+msg);
- } else if (!v.equals(ret))
- return type_inference(ret);
- } else return ret2;
- else return ret1;
- case `n:
- if (n.is_long())
- return #<int>;
- else if (n.is_double())
- return #<float>;
- else if (n.is_string())
- return #<string>;
- };
- type_error(e,"Wrong expression: "+print_query(e));
- throw new Error();
- }
-
- /** check the type for inconsistencies and fix the transient/persistent components */
- public static Tree normalize_type ( Tree type ) {
- match type {
- case record(...bs):
- Trees as = #[];
- Trees vs = #[];
- for ( Tree b: bs )
- match b {
- case bind(`v,`tp):
- if (v.is_variable())
- if (vs.member(v))
- type_error(type,"Duplicate record attributes: "+print_type(type));
- else {
- vs = vs.append(v);
- as = as.append(#<bind(`v,`(normalize_type(tp)))>);
- }
- else type_error(type,"Expected an attribute name: "+v);
- case _: type_error(type,"Ill-formed record type: "+print_type(type));
- };
- return #<record(...as)>;
- case tuple(...ts):
- Trees as = #[];
- for ( Tree t: ts )
- as = as.append(normalize_type(t));
- return #<tuple(...as)>;
- case union(...bs):
- Trees as = #[];
- for ( Tree b: bs )
- match b {
- case `c(`tp):
- as = as.append(#<`c(`(normalize_type(tp)))>);
- case _: type_error(type,"Ill-formed union type: "+print_type(type));
- };
- return #<union(...as)>;
- case persistent(bag(`tp)):
- Tree ntp = normalize_type(tp);
- return #<Bag(`ntp)>;
- case persistent(list(`tp)):
- Tree ntp = normalize_type(tp);
- return #<List(`ntp)>;
- case `T(`tp):
- if (is_collection(T))
- return #<`T(`(normalize_type(tp)))>;
- else fail
- case `tp:
- if (!tp.is_variable())
- fail;
- if (#[bool,byte,short,int,long,float,double,string].member(tp))
- return tp;
- Tree rt = global_datatype_env.lookup(tp.toString());
- if (rt != null)
- return tp;
- rt = type_names.lookup(tp.toString());
- if (rt != null)
- return tp;
- };
- type_error(type,"Unrecognized type: "+print_type(type));
- return type;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Union.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Union.java b/src/main/java/core/Union.java
deleted file mode 100644
index 540b6d9..0000000
--- a/src/main/java/core/Union.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** union values are tagged values, where tag is the descriminator */
-public class Union extends MRData {
- private byte tag;
- private MRData value;
-
- public Union ( byte tag, MRData value ) {
- this.tag = tag;
- this.value = value;
- }
-
- public void materializeAll () { value.materializeAll(); };
-
- public byte tag () { return tag; }
-
- public MRData value () { return value; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.UNION);
- out.writeByte(tag);
- value.write(out);
- }
-
- final public static Union read ( DataInput in ) throws IOException {
- return new Union(in.readByte(),MRContainer.read(in));
- }
-
- public void readFields ( DataInput in ) throws IOException {
- tag = in.readByte();
- value = MRContainer.read(in);
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof Union);
- Union p = (Union) x;
- return (tag == p.tag)
- ? value.compareTo(p.value)
- : (tag - p.tag);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- int k = (x[xs] == y[ys]) ? MRContainer.compare(x,xs+1,xl-1,y,ys+1,yl-1,size) : (x[xs]-y[ys]);
- size[0] += 2;
- return k;
- }
-
- public boolean equals ( Object x ) {
- return x instanceof Union && ((Union)x).tag==tag
- && ((Union)x).value.equals(value);
- }
-
- public int hashCode () {
- return Math.abs(tag ^ value.hashCode());
- }
-
- public String toString () {
- return "union("+tag+","+value+")";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/XMLParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/XMLParser.java b/src/main/java/core/XMLParser.java
deleted file mode 100644
index a25acbd..0000000
--- a/src/main/java/core/XMLParser.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import javax.xml.parsers.SAXParserFactory;
-import org.xml.sax.*;
-import java.io.*;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** An XML parser */
-final public class XMLParser implements Parser {
- String[] tags; // split tags
- Tree xpath; // XPath query for fragmentation
- XMLSplitter splitter;
- XPathParser parser;
- XMLReader xmlReader;
- final static SAXParserFactory factory = SAXParserFactory.newInstance();
-
- public void initialize ( Trees args ) {
- try {
- if (args.length() > 0) {
- if (!(args.nth(0) instanceof Node)
- || !(((Node)args.nth(0)).name().equals("list")
- || ((Node)args.nth(0)).name().equals("bag")))
- throw new Error("Expected a bag of synchronization tagnames to split the XML source: "+args.nth(0));
- Trees ts = ((Node)args.nth(0)).children();
- if (ts.length() == 0)
- throw new Error("Expected at least one synchronization tagname in XML source: "+ts);
- tags = new String[ts.length()];
- for ( int i = 0; i < tags.length; i++ )
- if (ts.nth(i) instanceof StringLeaf)
- tags[i] = ((StringLeaf)(ts.nth(i))).value();
- else throw new Error("Expected a synchronization tagname in XML source: "+ts.nth(i));
- if (args.length() == 2)
- xpath = ((Node)args.nth(1)).children().nth(0);
- else xpath = new VariableLeaf("dot");
- } else xpath = new VariableLeaf("dot");
- parser = new XPathParser(xpath);
- factory.setValidating(false);
- factory.setNamespaceAware(false);
- xmlReader = factory.newSAXParser().getXMLReader();
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public void open ( String file ) {
- try {
- splitter = new XMLSplitter(tags,file,new DataOutputBuffer());
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public void open ( FSDataInputStream fsin, long start, long end ) {
- try {
- splitter = new XMLSplitter(tags,fsin,start,end,new DataOutputBuffer());
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public Tree type () { return new VariableLeaf("XML"); }
-
- public String slice () {
- if (splitter.hasNext()) {
- DataOutputBuffer b = splitter.next();
- return new String(b.getData(),0,b.getLength());
- } else return null;
- }
-
- public Bag parse ( String s ) {
- try {
- parser.dataConstructor.start();
- xmlReader.setContentHandler(parser.handler);
- xmlReader.parse(new InputSource(new StringReader(s)));
- Bag b = new Bag();
- for ( MRData e: parser.dataConstructor.value() )
- b.add(e);
- return b;
- } catch (Exception e) {
- System.err.println(e);
- return new Bag();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/XMLSplitter.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/XMLSplitter.java b/src/main/java/core/XMLSplitter.java
deleted file mode 100644
index e407d85..0000000
--- a/src/main/java/core/XMLSplitter.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.Iterator;
-import javax.xml.parsers.SAXParserFactory;
-import org.xml.sax.*;
-import java.io.*;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.fs.FSDataInputStream;
-
-
-/** Extract the XML elements tagged by tags from a data split of the input stream (fsin)
- * and store them in a buffer (to be parsed by SAX).
- */
-final public class XMLSplitter implements Iterator<DataOutputBuffer> {
- boolean in_memory;
- FSDataInputStream fsin; // for HDFS processing
- BufferedReader in; // for in-memory processing
- String[] tags;
- long start;
- long end;
- StringBuffer tagname = new StringBuffer(100);
- String start_tagname;
- final DataOutputBuffer buffer;
-
- XMLSplitter ( String[] tags, FSDataInputStream fsin, long start, long end,
- DataOutputBuffer buffer ) {
- in_memory = false;
- this.tags = tags;
- this.fsin = fsin;
- this.start = start;
- this.end = end;
- this.buffer = buffer;
- try {
- fsin.seek(start);
- } catch ( IOException e ) {
- System.err.println("*** Cannot parse the data split: "+fsin);
- }
- }
-
- XMLSplitter ( String[] tags, String file, DataOutputBuffer buffer ) {
- in_memory = true;
- try {
- in = new BufferedReader(new InputStreamReader(new FileInputStream(file)),
- 100000);
- } catch ( Exception e ) {
- throw new Error("Cannot open the file: "+file);
- };
- this.tags = tags;
- this.buffer = buffer;
- }
-
- public boolean hasNext () {
- try {
- if (in_memory || fsin.getPos() < end)
- if (skip())
- return store();
- return false;
- } catch (Exception e) {
- System.err.println(e);
- return false;
- }
- }
-
- public DataOutputBuffer next () {
- return buffer;
- }
-
- public void remove () { }
-
- boolean is_start_tag () {
- if (tags == null)
- return true;
- for (String tag: tags)
- if (tag.contentEquals(tagname))
- return true;
- return false;
- }
-
- char read_tag () throws IOException {
- tagname.setLength(0);
- while (true) {
- int b = in_memory ? in.read() : fsin.read();
- if (b == -1)
- return ' ';
- else if (!Character.isLetterOrDigit(b) && b != ':' && b != '_')
- return (char)b;
- tagname.append((char)b);
- }
- }
-
- /** skip until the beginning of a split element */
- boolean skip () throws IOException {
- while (true) {
- int b = in_memory ? in.read() : fsin.read();
- if (b == -1 || (!in_memory && fsin.getPos() >= end))
- return false;
- else if (b == '<') {
- b = read_tag();
- if (is_start_tag()) {
- buffer.reset();
- buffer.write('<');
- for ( int i = 0; i < tagname.length(); i++ )
- buffer.write(tagname.charAt(i));
- buffer.write(b);
- start_tagname = new String(tagname);
- return true;
- }
- }
- }
- }
-
- /** store one split element into the buffer; may cross split boundaries */
- boolean store () throws IOException {
- while (true) {
- int b = in_memory ? in.read() : fsin.read();
- if (b == -1)
- return false;
- if (b == '&') { // don't validate external XML entities
- buffer.write('&');buffer.write('a');buffer.write('m');buffer.write('p');buffer.write(';');
- } else buffer.write(b);
- if (b == '<') {
- b = in_memory ? in.read() : fsin.read();
- buffer.write(b);
- if (b == '/') {
- b = read_tag();
- for ( int i = 0; i < tagname.length(); i++ )
- buffer.write(tagname.charAt(i));
- buffer.write(b);
- if (start_tagname.contentEquals(tagname)) {
- while (b != '>') {
- b = fsin.read();
- buffer.write(b);
- };
- return true;
- }
- }
- }
- }
- }
-}
[09/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Evaluator.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Evaluator.java b/src/main/java/core/Evaluator.java
deleted file mode 100644
index 28d356f..0000000
--- a/src/main/java/core/Evaluator.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java_cup.runtime.*;
-import org.apache.mrql.gen.*;
-import java.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-
-
-/** Evaluates physical plans using one of the evaluation engines */
-abstract public class Evaluator extends Interpreter {
-
- /** the current MRQL evaluator */
- public static Evaluator evaluator;
-
- /** initialize the evaluator */
- abstract public void init ( Configuration conf );
-
- /** shutdown the evaluator */
- abstract public void shutdown ( Configuration conf );
-
- /** initialize the query evaluation */
- abstract public void initialize_query ();
-
- /** create a new evaluation configuration */
- abstract public Configuration new_configuration ();
-
- /** synchronize peers in BSP mode */
- public MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
- throw new Error("You can only synchronize BSP tasks");
- }
-
- /** distribute a bag among peers in BSP mode */
- public Bag distribute ( MR_string peerName, Bag s ) {
- throw new Error("You can only distribute bags among BSP tasks");
- }
-
- /** run a BSP task */
- public MRData bsp ( Tree plan, Environment env ) throws Exception {
- throw new Error("You can only run a BSP task in BSP mode");
- }
-
- /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
- abstract public Class<? extends MRQLFileInputFormat> parsedInputFormat ();
-
- /** return the FileInputFormat for binary files */
- abstract public Class<? extends MRQLFileInputFormat> binaryInputFormat ();
-
- /** return the FileInputFormat for data generator files */
- abstract public Class<? extends MRQLFileInputFormat> generatorInputFormat ();
-
- /** The Aggregate physical operator
- * @param acc_fnc the accumulator function from (T,T) to T
- * @param zero the zero element of type T
- * @param plan the plan that constructs the dataset that contains the bag of values {T}
- * @param env contains bindings fro variables to values (MRData)
- * @return the aggregation result of type T
- */
- abstract public MRData aggregate ( Tree acc_fnc,
- Tree zero,
- Tree plan,
- Environment env ) throws Exception;
-
- /** Evaluate a loop a fixed number of times */
- abstract public Tuple loop ( Tree e, Environment env ) throws Exception;
-
- /** Evaluate a MRQL physical plan and print tracing info
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a DataSet (stored in HDFS)
- */
- abstract public DataSet eval ( final Tree e,
- final Environment env,
- final String counter );
-
- final static MR_long counter_key = new MR_long(0);
- final static MRContainer counter_container = new MRContainer(counter_key);
- final static MRContainer value_container = new MRContainer(new MR_int(0));
-
- /** dump MRQL data into a sequence file */
- public void dump ( String file, Tree type, MRData data ) throws Exception {
- Path path = new Path(file);
- FileSystem fs = path.getFileSystem(Plan.conf);
- PrintStream ftp = new PrintStream(fs.create(path.suffix(".type")));
- ftp.print("2@"+type.toString()+"\n");
- ftp.close();
- SequenceFile.Writer writer
- = new SequenceFile.Writer(fs,Plan.conf,path,
- MRContainer.class,MRContainer.class);
- if (data instanceof MR_dataset)
- data = Plan.collect(((MR_dataset)data).dataset());
- if (data instanceof Bag) {
- Bag s = (Bag)data;
- long i = 0;
- for ( MRData e: s ) {
- counter_key.set(i++);
- value_container.set(e);
- writer.append(counter_container,value_container);
- }
- } else {
- counter_key.set(0);
- value_container.set(data);
- writer.append(counter_container,value_container);
- };
- writer.close();
- }
-
- /** dump MRQL data into a text CVS file */
- public void dump_text ( String file, Tree type, MRData data ) throws Exception {
- int ps = Config.max_bag_size_print;
- Config.max_bag_size_print = -1;
- final PrintStream out = (Config.hadoop_mode)
- ? Plan.print_stream(file)
- : new PrintStream(file);
- if (data instanceof MR_dataset)
- data = Plan.collect(((MR_dataset)data).dataset());
- if (Translator.collection_type(type)) {
- Tree tp = ((Node)type).children().head();
- if (tp instanceof Node && ((Node)tp).name().equals("tuple")) {
- Trees ts = ((Node)tp).children();
- for ( MRData x: (Bag)data ) {
- Tuple t = (Tuple)x;
- out.print(print(t.get((short)0),ts.nth(0)));
- for ( short i = 1; i < t.size(); i++ )
- out.print(","+print(t.get(i),ts.nth(i)));
- out.println();
- }
- } else for ( MRData x: (Bag)data )
- out.println(print(x,tp));
- } else out.println(print(data,query_type));
- Config.max_bag_size_print = ps;
- out.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Function.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Function.java b/src/main/java/core/Function.java
deleted file mode 100644
index 1f1d4e1..0000000
--- a/src/main/java/core/Function.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-
-
-/**
- * An anonymous function from MRData to MRData (a lambda abstraction)
- * Must provide a concrete implementation for eval (the lambda body)
- */
-abstract public class Function implements Serializable {
- /**
- * Evaluate the anonymous function from MRData to MRData
- * @param arg the operand to be evaluated
- * @return the result of evaluating
- */
- abstract public MRData eval ( final MRData arg );
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/GeneratorDataSource.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/GeneratorDataSource.java b/src/main/java/core/GeneratorDataSource.java
deleted file mode 100644
index 380f50d..0000000
--- a/src/main/java/core/GeneratorDataSource.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** A DataSource used for processing the range min..max */
-final public class GeneratorDataSource extends DataSource {
- GeneratorDataSource ( int source_num, String path, Configuration conf ) {
- super(source_num,path,Evaluator.evaluator.generatorInputFormat(),conf);
- }
-
- GeneratorDataSource ( String path, Configuration conf ) {
- super(-1,path,Evaluator.evaluator.generatorInputFormat(),conf);
- }
-
- public static long size ( Path path, Configuration conf ) throws IOException {
- // each file generates range_split_size long integers
- FileStatus s = path.getFileSystem(conf).getFileStatus(path);
- if (!s.isDir())
- return Config.range_split_size*8;
- long size = 0;
- for ( FileStatus fs: path.getFileSystem(conf).listStatus(path) )
- size += Config.range_split_size*8;
- return size;
- }
-
- public String toString () {
- return "Generator"+separator+source_num+separator+path;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Interpreter.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Interpreter.gen b/src/main/java/core/Interpreter.gen
deleted file mode 100644
index 97c76ce..0000000
--- a/src/main/java/core/Interpreter.gen
+++ /dev/null
@@ -1,833 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.io.PrintStream;
-
-
-/** The MRQL interpreter */
-public class Interpreter extends TypeInference {
-
- public final static Tree identity_mapper = #<lambda(x,bag(x))>;
-
- protected static Environment global_env = null;
-
- /** retrieve variable binding */
- public final static MRData variable_lookup ( final String v, final Environment environment ) {
- for ( Environment env = environment; env != null; env = env.next ) {
- if (v.equals(env.name))
- return env.value;
- };
- return null;
- }
-
- /** insert a new global variable binding */
- public final static void new_global_binding ( final String var, final MRData value ) {
- if (value instanceof Bag)
- ((Bag)value).materialize();
- global_env = new Environment(var,value,global_env);
- }
-
- /** remove a global variable binding */
- public static void remove_global_binding ( String v ) {
- if (global_env == null)
- return;
- for ( Environment env = global_env; env.next != null; env = env.next )
- if (v.equals(env.next.name))
- env.next = env.next.next;
- if (global_env.name == v)
- global_env = global_env.next;
- }
-
- /** retrieve a global variable binding */
- public static MRData lookup_global_binding ( String v ) {
- for ( Environment env = global_env; env != null; env = env.next )
- if (v.equals(env.name))
- return env.value;
- return null;
- }
-
- public static void set_global_bindings ( Environment env ) {
- global_env = env;
- }
-
- final static int coerce_method = ClassImporter.find_method_number("coerce",#[any,int]);
-
- /** untyped reify: not type-correct but will not crash the run-time system */
- private final static Tree reify ( final MRData x ) {
- if (x instanceof Bag) {
- Bag b = (Bag)x;
- Trees as = #[];
- for ( MRData e: b)
- as = as.append(reify(e));
- return #<list(...as)>;
- } else if (x instanceof Tuple) {
- Tuple t = (Tuple)x;
- Trees as = #[];
- for ( short i = 0; i < t.size(); i++ )
- as = as.append(reify(t.get(i)));
- return #<tuple(...as)>;
- } else if (x instanceof MR_string)
- return new StringLeaf(((MR_string)x).get());
- else if (x instanceof MR_short)
- return #<callM(coerce,`coerce_method,`(((MR_short)x).get()),`(MRContainer.SHORT))>;
- else if (x instanceof MR_int)
- return #<`(((MR_int)x).get())>;
- else if (x instanceof MR_long)
- return #<callM(coerce,`coerce_method,`((int)((MR_long)x).get()),`(MRContainer.LONG))>;
- else if (x instanceof MR_float)
- return #<`(((MR_float)x).get())>;
- else if (x instanceof MR_double)
- return #<callM(coerce,`coerce_method,`((float)(((MR_double)x).get())),`(MRContainer.DOUBLE))>;
- throw new Error("wrong MRData: "+x);
- }
-
- /** evaluate an MRQL function in memory */
- private final static Function evalf ( final String v,
- final Tree body,
- final Environment env ) {
- return new Function() {
- final public MRData eval ( final MRData x ) {
- return evalE(body,new Environment(v,x,env));
- }
- };
- }
-
- /** evaluate an MRQL function in memory */
- public final static Function evalF ( Tree fnc, Environment env ) {
- match fnc {
- case compiled(`ln,`lm,...vars):
- try {
- return Compiler.compiled(Thread.currentThread().getContextClassLoader(),ln.toString());
- } catch (Exception ex) {
- System.err.println("*** Unable to retrieve the compiled lambda: "+fnc);
- return ((Lambda) evalE(lm)).lambda();
- }
- case lambda(`v,`b):
- return evalf(v.toString(),b,env);
- case function(tuple(...params),`tp,`body):
- String[] as = new String[params.length()];
- int i = 0;
- for ( Tree param: params )
- match param {
- case `bind(`v,_):
- as[i++] = v.toString();
- };
- return evalT(as,body,env);
- };
- throw new Error("Ill-formed lambda: "+fnc);
- }
-
- /** evaluate an MRQL function in memory */
- private final static Function evalT ( final String[] params,
- final Tree body,
- final Environment env ) {
- return new Function() {
- final public MRData eval ( final MRData x ) {
- Environment new_env = env;
- for ( int i = 0; i < params.length; i++ )
- new_env = new Environment(params[i],((Tuple)x).get(i),new_env);
- return evalE(body,new_env);
- }
- };
- }
-
- final static String true_name = #<true>.toString();
- final static String false_name = #<false>.toString();
- final static String null_name = #<null>.toString();
- final static MRData null_value = new Tuple(0);
- final static MRData true_value = new MR_bool(true);
- final static MRData false_value = new MR_bool(false);
-
- static int tab_count = -3;
-
- public static String tabs ( int n ) {
- StringBuffer b = new StringBuffer();
- for ( int i = 0; i < n; i++)
- b.append(' ');
- return b.toString();
- }
-
- /** evaluate an MRQL expression in memory and print tracing info */
- final static MRData evalE ( final Tree e, final Environment env ) {
- if (Config.trace_exp_execution) {
- tab_count += 3;
- System.out.println(tabs(tab_count)+print_query(e));
- };
- MRData res = evalEE(e,env);
- if (Config.trace_exp_execution) {
- System.out.println(tabs(tab_count)+"-> "+res);
- tab_count -= 3;
- };
- return res;
- }
-
- /** evaluate an MRQL expression in memory */
- private final static MRData evalEE ( final Tree e, final Environment env ) {
- try {
- if (e.is_variable()) {
- String v = e.toString();
- if (v == true_name)
- return true_value;
- else if (v == false_name)
- return false_value;
- else if (v == null_name)
- return null_value;
- MRData x = variable_lookup(v,env);
- if (x != null)
- return x;
- x = lookup_global_binding(v);
- if (x == null)
- throw new Error("Variable "+v+" is not bound");
- return x;
- } else if (e.is_long())
- return new MR_int((int)e.longValue());
- else if (e.is_double())
- return new MR_float((float)e.doubleValue());
- else if (e.is_string())
- return new MR_string(e.stringValue());
- match e {
- case callM(and,_,`x,`y): // lazy
- return (((MR_bool)evalE(x,env)).get()) ? evalE(y,env) : false_value;
- case callM(or,_,`x,`y):
- return (((MR_bool)evalE(x,env)).get()) ? true_value : evalE(y,env);
- case callM(`f,`n,...args): // internal function call
- MRData[] as = new MRData[args.length()];
- for ( int i = 0; i < args.length(); i++ )
- as[i] = evalE(args.nth(i),env);
- return ClassImporter.call((int)n.longValue(),as);
- case compiled(`ln,_,...vars):
- return new Lambda(Compiler.compiled(Thread.currentThread().getContextClassLoader(),ln.toString()));
- case lambda(`v,`body):
- return new Lambda(evalf(v.toString(),body,env));
- case nth(`x,`n):
- return ((Tuple)evalE(x,env)).get((int)n.longValue());
- case setNth(`x,`n,`v,`ret):
- return ((Tuple)evalE(x,env)).set((int)n.longValue(),evalE(v,env),evalE(ret,env));
- case materialize(`u):
- return MapReduceAlgebra.materialize(evalE(u,env));
- case let(`v,`u,`body):
- MRData x = evalE(u,env);
- if (x instanceof Bag)
- ((Bag)x).materialize();
- return evalE(body,new Environment(v.toString(),x,env));
- case cmap(`f,`s):
- return MapReduceAlgebra.cmap(evalF(f,env),(Bag)evalE(s,env));
- case filter(`p,`m,`s):
- return MapReduceAlgebra.filter(evalF(p,env),evalF(m,env),(Bag)evalE(s,env));
- case map(`m,`s):
- return MapReduceAlgebra.map(evalF(m,env),(Bag)evalE(s,env));
- case repeat(lambda(`v,`b),`s,`n):
- final String nm = v.toString();
- final Tree body = b;
- if (Config.hadoop_mode) {
- Function loop_fnc = new Function () {
- public MRData eval ( MRData s ) {
- new_global_binding(nm,s);
- MRData val = new MR_dataset(Evaluator.evaluator.eval(body,new Environment(nm,s,env),nm));
- return val;
- }; };
- return MapReduceAlgebra.repeat(loop_fnc,(Bag)evalE(s,env),((MR_int)evalE(n,env)).get());
- } else {
- Function loop_fnc = new Function () {
- public MRData eval ( MRData s ) {
- new_global_binding(nm,s);
- return evalM(body,new Environment(nm,s,env));
- }; };
- return MapReduceAlgebra.repeat(loop_fnc,(Bag)evalE(s,env),((MR_int)evalE(n,env)).get());
- }
- case repeat(`lm,`s,`n):
- return MapReduceAlgebra.repeat(evalF(lm,env),(Bag)evalE(s,env),((MR_int)evalE(n,env)).get());
- case range(`min,`max):
- return MapReduceAlgebra.generator(((MR_long)evalE(min,env)).get(),
- ((MR_long)evalE(max,env)).get());
- case call(`f,...args):
- Tuple t = new Tuple(args.length());
- int i = 0;
- for ( Tree a: args )
- t.set(i++,evalE(a,env));
- return evalF(f,env).eval(t);
- case tuple(`x,`y):
- return new Tuple(evalE(x,env),evalE(y,env));
- case tuple(`x,`y,`z):
- return new Tuple(evalE(x,env),evalE(y,env),evalE(z,env));
- case tuple(...el):
- Tuple t = new Tuple(el.length());
- int i = 0;
- for ( Tree a: el )
- t.set(i++,evalE(a,env));
- return t;
- case tagged_union(`n,`u):
- return new Union((byte)n.longValue(),evalE(u,env));
- case union_value(`x):
- return ((Union)evalE(x,env)).value();
- case union_tag(`x):
- return new MR_int(((Union)evalE(x,env)).tag());
- // used for shortcutting sync in bsp supersteps
- case BAG():
- return SystemFunctions.bsp_empty_bag;
- case TRUE():
- return SystemFunctions.bsp_true_value;
- case FALSE():
- return SystemFunctions.bsp_false_value;
- case `T(...el):
- if (!is_collection(T))
- fail;
- if (el.is_empty())
- return new Bag();
- Bag b = new Bag(el.length());
- for ( Tree a: el )
- b.add(evalE(a,env));
- return b;
- case if(`c,`x,`y):
- if (((MR_bool)evalE(c,env)).get())
- return evalE(x,env);
- else return evalE(y,env);
- case Collect(`s):
- try {
- if (Config.hadoop_mode)
- return Plan.collect(Evaluator.evaluator.eval(s,env,"-"));
- Bag b = evalS(s,env);
- b.materialize();
- return b;
- } catch (Exception ex) { throw new Error(ex); }
- case dataset_size(`x):
- return new MR_long(Plan.size(Evaluator.evaluator.eval(x,env,"-")) / (1024*1024));
- case synchronize(`peer,`b):
- return Evaluator.evaluator.synchronize(((MR_string)evalE(peer,env)),(MR_bool)evalE(b,env));
- case distribute(`peer,`s):
- return Evaluator.evaluator.distribute(((MR_string)evalE(peer,env)),(Bag)evalE(s,env));
- case mapReduce(`m,`r,`s,_):
- return MapReduceAlgebra.mapReduce(evalF(m,env),
- evalF(r,env),
- (Bag)evalE(s,env));
- case mapReduce2(`mx,`my,`r,`x,`y,_):
- return MapReduceAlgebra.mapReduce2(
- evalF(mx,env),
- evalF(my,env),
- evalF(r,env),
- (Bag)evalE(x,env),
- (Bag)evalE(y,env));
- case mapJoin(`kx,`ky,`r,`x,`y):
- return MapReduceAlgebra.mapJoin(
- evalF(kx,env),
- evalF(ky,env),
- evalF(r,env),
- (Bag)evalE(x,env),
- (Bag)evalE(y,env));
- case crossProduct(`mx,`my,`r,`x,`y):
- return MapReduceAlgebra.crossProduct(
- evalF(mx,env),
- evalF(my,env),
- evalF(r,env),
- (Bag)evalE(x,env),
- (Bag)evalE(y,env));
- case groupBy(`s):
- return MapReduceAlgebra.groupBy((Bag)evalE(s,env));
- case orderBy(`s):
- return MapReduceAlgebra.groupBy((Bag)evalE(s,env));
- case index(`x,`n):
- MRData xv = evalE(x,env);
- MRData nv = evalE(n,env);
- final int k = (int)((MR_int)nv).get();
- if (k < 0)
- throw new Error("Negative list index: "+k);
- if (xv instanceof MR_dataset) {
- List<MRData> res = ((MR_dataset)xv).dataset().take(k+1);
- if (k >= res.size())
- throw new Error("List index out of bounds: "+k);
- return res.get(k);
- };
- Bag b = (Bag)xv;
- return b.get(k);
- case range(`x,`i,`j):
- MRData xv = evalE(x,env);
- MRData ni = evalE(i,env);
- MRData nj = evalE(j,env);
- int ki = (int)((MR_int)ni).get();
- int kj = (int)((MR_int)nj).get();
- if (ki < 0 || kj < ki)
- throw new Error("Wrong list range: ["+ki+","+kj+"]");
- Iterator<MRData> it = (xv instanceof MR_dataset)
- ? ((MR_dataset)xv).dataset().take(kj+1).iterator()
- : ((Bag)xv).iterator();
- Bag s = new Bag();
- for ( int n = 0; it.hasNext() && n < ki; n++ )
- it.next();
- for ( int n = ki; it.hasNext() && n <= kj; n++ )
- s.add(it.next());
- return s;
- case map_index(`x,`key):
- MRData xv = evalE(x,env);
- final MRData nk = evalE(key,env);
- if (xv instanceof MR_dataset) {
- xv = ((MR_dataset)xv).dataset().reduce(new Tuple(),new Function() {
- public MRData eval ( MRData value ) {
- Tuple p = (Tuple)value;
- Tuple y = (Tuple)p.second();
- return (y.first().equals(nk)) ? y.second() : p.first();
- }
- });
- if (xv instanceof Tuple && ((Tuple)xv).size() == 0)
- throw new Error("Map key not found: "+nk);
- return xv;
- };
- return ((Bag)xv).map_find(nk);
- case aggregate(`acc,`zero,`s):
- return MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
- (Bag)evalE(s,env));
- case Aggregate(`acc,`zero,`s):
- if (Config.hadoop_mode)
- return Evaluator.evaluator.aggregate(closure(acc,env),zero,s,env);
- else return MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),evalM(s,env));
- case mergeGroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
- return MapReduceAlgebra.mergeGroupByJoin(evalF(kx,env),evalF(ky,env),evalF(gx,env),evalF(gy,env),
- evalF(m,env),evalF(c,env),evalF(r,env),
- (Bag)evalE(x,env),(Bag)evalE(y,env));
- case BSP(tuple(...ns),`superstep,`state,`o,...as):
- if (Config.hadoop_mode)
- return Evaluator.evaluator.bsp(e,env);
- Bag[] ds = new Bag[as.length()];
- for ( int i = 0; i < ds.length; i++ )
- ds[i] = evalM(as.nth(i),env);
- int[] nn = new int[ns.length()];
- for ( int i = 0; i < ns.length(); i++ )
- nn[i] = (int)((LongLeaf)ns.nth(i)).value();
- return MapReduceAlgebra.BSP(nn,
- evalF(superstep,env),
- evalE(state,env),
- o.equals(#<true>),
- ds);
- case BSP(`n,`superstep,`state,`o,...as):
- if (Config.hadoop_mode)
- return Evaluator.evaluator.bsp(e,env);
- Bag[] ds = new Bag[as.length()];
- for ( int i = 0; i < ds.length; i++ )
- ds[i] = evalM(as.nth(i),env);
- return MapReduceAlgebra.BSP(new int[]{(int)((LongLeaf)n).value()},
- evalF(superstep,env),
- evalE(state,env),
- o.equals(#<true>),
- ds);
- case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`num):
- if (Config.hadoop_mode)
- return Evaluator.evaluator.loop(e,env);
- int limit = ((MR_int)evalE(num,env)).get();
- Bag[] s = new Bag[vs.length()];
- for ( int i = 0; i < vs.length(); i++ )
- s[i] = evalM(ss.nth(i),env);
- for ( int n = 0; n < limit; n++ ) {
- Environment nenv = env;
- for ( int i = 0; i < vs.length(); i ++ ) {
- s[i].materialize();
- nenv = new Environment(vs.nth(i).toString(),s[i],nenv);
- };
- for ( int i = 0; i < vs.length(); i ++ )
- s[i] = (Bag)evalM(bs.nth(i),nenv);
- };
- return new Tuple(s);
- case function(tuple(...params),`tp,`body):
- String[] as = new String[params.length()];
- int i = 0;
- for ( Tree param: params )
- match param {
- case `bind(`v,_):
- as[i++] = v.toString();
- };
- return new Lambda(evalT(as,body,env));
- case typed(`x,_):
- return evalE(x,env);
- case apply(`f,`arg):
- if (!f.is_variable())
- return evalF(f,env).eval(evalE(arg,env));
- MRData fnc = lookup_global_binding(f.toString());
- if (fnc == null) {
- String s = Plan.conf.get("mrql.global."+f);
- if (s != null)
- try {
- Tree ft = Tree.parse(s);
- TopLevel.store(f.toString(),ft);
- fnc = evalE(ft,env);
- new_global_binding(f.toString(),fnc);
- } catch (Exception ex) {
- throw new Error(ex);
- }
- };
- MRData t = evalE(arg,env);
- if (!(t instanceof Tuple))
- throw new Error("Expected a tuple in function application: "+t);
- return ((Lambda)fnc).lambda().eval(t);
- case trace(`x):
- MRData z = evalE(x,env);
- System.err.println("*** "+x+": "+z);
- return z;
- case _:
- try {
- if (Config.hadoop_mode)
- return new MR_dataset(Evaluator.evaluator.eval(e,env,"-"));
- else return evalS(e,env);
- } catch (Exception ex) { throw new Error(ex); }
- };
- throw new Error("Cannot evaluate the expression: "+e);
- } catch (Error msg) {
- if (!Config.trace)
- throw new Error(msg.getMessage());
- System.err.println(msg.getMessage());
- msg.printStackTrace();
- throw new Error("Evaluation error in: "+print_query(e));
- } catch (Exception ex) {
- if (Config.trace) {
- System.err.println(ex.getMessage());
- ex.printStackTrace();
- }
- throw new Error("Evaluation error in: "+print_query(e));
- }
- }
-
- /** evaluate an MRQL expression in memory */
- final static MRData evalE ( final Tree e ) {
- return evalE(e,null);
- }
-
- /** evaluate MRQL physical operators in memory (returns a Bag) */
- final static Bag evalS ( final Tree e, final Environment env ) {
- return evalM(e,env);
- }
-
- /** evaluate MRQL physical operators in memory (returns a Bag) */
- final static Bag evalM ( final Tree e, final Environment env ) {
- if (Config.trace_execution) {
- tab_count += 3;
- System.out.println(tabs(tab_count)+print_query(e));
- };
- Bag res = evalMM(e,env);
- if (Config.trace_execution) {
- System.out.println(tabs(tab_count)+"-> "+res);
- tab_count -= 3;
- };
- return res;
- }
-
- /** evaluate MRQL physical operators in memory (returns a Bag) */
- final static Bag evalMM ( final Tree e, final Environment env ) {
- try {
- match e {
- case cMap(`f,`s):
- return MapReduceAlgebra.cmap(evalF(f,env),evalM(s,env));
- case AggregateMap(`f,`acc,`zero,`s):
- return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
- evalM(#<cMap(`f,`s)>,env)));
- case MapReduce(`m,`r,`s,_):
- return MapReduceAlgebra.mapReduce(
- evalF(m,env),
- evalF(r,env),
- evalM(s,env));
- case MapAggregateReduce(`m,`r,`acc,`zero,`s,_):
- return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
- evalM(#<MapReduce(`m,`r,`s,false)>,env)));
- case MapCombineReduce(`m,`c,`r,`s,_):
- return MapReduceAlgebra.mapReduce(
- evalF(m,env),
- evalF(r,env),
- evalM(s,env));
- case MapReduce2(`mx,`my,`r,`x,`y,_):
- return MapReduceAlgebra.mapReduce2(
- evalF(mx,env),
- evalF(my,env),
- evalF(r,env),
- evalM(x,env),
- evalM(y,env));
- case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,_):
- return MapReduceAlgebra.mapReduce2(
- evalF(mx,env),
- evalF(my,env),
- evalF(r,env),
- evalM(x,env),
- evalM(y,env));
- case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,_):
- return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
- evalM(#< MapReduce2(`mx,`my,`r,`x,`y,false)>,env)));
- case MapJoin(`kx,`ky,`r,`x,`y):
- return MapReduceAlgebra.mapJoin(
- evalF(kx,env),
- evalF(ky,env),
- evalF(r,env),
- evalM(x,env),
- evalM(y,env));
- case MapAggregateJoin(`kx,`ky,`r,`acc,`zero,`x,`y):
- return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
- evalM(#<MapJoin(`kx,`ky,`r,`x,`y)>,env)));
- case GroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
- return MapReduceAlgebra.groupByJoin(
- evalF(kx,env),
- evalF(ky,env),
- evalF(gx,env),
- evalF(gy,env),
- evalF(m,env),
- evalF(c,env),
- evalF(r,env),
- evalM(x,env),
- evalM(y,env));
- case CrossProduct(`mx,`my,`r,`x,`y):
- return MapReduceAlgebra.crossProduct(
- evalF(mx,env),
- evalF(my,env),
- evalF(r,env),
- evalM(x,env),
- evalM(y,env));
- case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
- return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
- evalM(#<CrossProduct(`mx,`my,`r,`x,`y)>,env)));
- case BinarySource(`file,_):
- return (Bag)MapReduceAlgebra.read_binary(file.stringValue());
- case BSPSource(`n,BinarySource(`file,_)):
- return (Bag)MapReduceAlgebra.read_binary((int)((LongLeaf)n).value(),
- file.stringValue());
- case BSPSource(`n,ParsedSource(`parser,`file,...args)):
- if (!(n instanceof LongLeaf))
- fail;
- Parser p = DataSource.parserDirectory.get(parser.toString()).newInstance();
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- return MapReduceAlgebra.parsedSource((int)(((LongLeaf)n).value()),p,
- ((MR_string)evalE(file,env)).get(),args);
- case ParsedSource(`parser,`file,...args):
- Parser p = DataSource.parserDirectory.get(parser.toString()).newInstance();
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- return MapReduceAlgebra.parsedSource(p,((MR_string)evalE(file,env)).get(),args);
- case Merge(`x,`y):
- return evalM(x,env).union(evalM(y,env));
- case Repeat(lambda(`v,`b),`s,`n):
- final String vs = v.toString();
- final Tree body = b;
- Function loop = new Function() {
- final public MRData eval ( final MRData x ) {
- return evalM(body,new Environment(vs,x,env));
- }
- };
- return MapReduceAlgebra.repeat(loop,(Bag)evalM(s,env),((MR_int)evalE(n,env)).get());
- case Closure(lambda(`v,`b),`s,`n):
- final String vs = v.toString();
- final Tree body = b;
- Function loop = new Function() {
- final public MRData eval ( final MRData x ) {
- return evalM(body,new Environment(vs,x,env));
- }
- };
- return MapReduceAlgebra.closure(loop,(Bag)evalM(s,env),((MR_int)evalE(n,env)).get());
- case Generator(`min,`max,`size):
- return MapReduceAlgebra.generator(((MR_long)evalE(min,env)).get(),
- ((MR_long)evalE(max,env)).get());
- case BSPSource(`n,Generator(`min,`max,`size)):
- return MapReduceAlgebra.generator((int)((LongLeaf)n).value(),
- ((MR_long)evalE(min,env)).get(),
- ((MR_long)evalE(max,env)).get());
- case Dump(`s):
- Bag bs = (Bag)evalE(s,env);
- final Iterator<MRData> iter = bs.iterator();
- return new Bag(new BagIterator() {
- public boolean hasNext () {
- return iter.hasNext();
- }
- public MRData next () {
- return new Tuple(new MR_int(0),iter.next());
- }
- });
- case let(`v,`u,`body):
- return evalM(body,new Environment(v.toString(),evalE(u,env),env));
- case apply(`f,`arg):
- if (!f.is_variable())
- return (Bag)evalF(f,env).eval(evalE(arg));
- MRData fnc = lookup_global_binding(f.toString());
- if (fnc == null)
- throw new Error("Unknown function: "+f);
- MRData t = evalE(arg,env);
- if (!(t instanceof Tuple))
- throw new Error("Expected a tuple in function application: "+t);
- return (Bag)((Lambda)fnc).lambda().eval(t);
- case BSPSource(`n,`s):
- final MR_int i = new MR_int((int)((LongLeaf)n).value());
- Bag bs = (Bag)evalE(s,env);
- final Iterator<MRData> iter = bs.iterator();
- return new Bag(new BagIterator() {
- public boolean hasNext () {
- return iter.hasNext();
- }
- public MRData next () {
- return new Tuple(i,iter.next());
- }
- });
- case BSP(...):
- MRData res = evalE(e,env);
- if (res instanceof Bag)
- return (Bag)res;
- else return new Bag(res);
- case `v:
- if (!v.is_variable())
- fail;
- MRData x = variable_lookup(v.toString(),env);
- if (x != null)
- return (Bag)x;
- x = lookup_global_binding(v.toString());
- if (x != null)
- return (Bag)x;
- throw new Error("Variable "+v+" is not bound");
- };
- throw new Error("Cannot evaluate the plan: "+e);
- } catch (Error msg) {
- if (!Config.trace)
- throw new Error(msg.getMessage());
- System.err.println(msg.getMessage());
- msg.printStackTrace();
- throw new Error("Evaluation error in: "+print_query(e));
- } catch (Exception ex) {
- if (Config.trace)
- ex.printStackTrace();
- throw new Error("Evaluation error in: "+print_query(e));
- }
- }
-
- /** replace all non-free variables with their reified values */
- private final static Tree closure ( Tree e, Environment env, Trees local_vars ) {
- match e {
- case lambda(`x,`b):
- return #<lambda(`x,`(closure(b,env,local_vars.cons(x))))>;
- case apply(`f,...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(closure(a,env,local_vars));
- return #<apply(`f,...bs)>;
- case `f(...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(closure(a,env,local_vars));
- return #<`f(...bs)>;
- case null: return null;
- case `v:
- if (!v.is_variable())
- fail;
- if (local_vars.member(v))
- fail;
- MRData x = variable_lookup(v.toString(),env);
- if (x != null)
- if (!(x instanceof MR_dataset))
- return reify(x);
- x = lookup_global_binding(v.toString());
- if (x != null)
- if (!(x instanceof MR_dataset))
- return reify(x);
- };
- return e;
- }
-
- /** replace all non-free variables with their reified values */
- final static Tree closure ( Tree e, Environment env ) {
- return closure(e,env,#[]);
- }
-
- static Tree query_type;
- static Tree query_plan;
- static boolean is_dataset;
-
- /** translate an MRQL expression e into a physical plan */
- final static Tree translate_expression ( Tree e ) {
- try {
- if (Config.trace)
- System.out.println("Query at line "+Main.parser.line_pos()+": "+print_query(e));
- Tree qt = TypeInference.type_inference(e);
- if (!Config.quiet_execution)
- System.out.println("Query type: "+print_type(qt));
- query_type = qt;
- Tree ne = Normalization.remove_groupby(e);
- if (Config.trace)
- System.out.println("After removing group-by:\n"+ne.pretty(0));
- ne = Simplification.rename(ne);
- if (Config.trace)
- System.out.println("After renaming variables:\n"+ne.pretty(0));
- ne = Simplification.rename(Normalization.normalize_all(ne));
- if (Config.trace)
- System.out.println("Normalized query:\n"+ne.pretty(0));
- type_inference(ne);
- ne = QueryPlan.best_plan(ne);
- if (Config.trace)
- System.out.println("Best plan:\n"+ne.pretty(0));
- ne = Simplification.rename(Translator.translate_select(ne));
- if (Config.trace)
- System.out.println("After removing select-queries:\n"+ne.pretty(0));
- type_inference(ne);
- ne = Simplification.simplify_all(ne);
- if (Config.trace)
- System.out.println("Algebra expression:\n"+ne.pretty(0));
- Tree pt = type_inference(ne);
- if (Config.trace)
- System.out.println("Algebraic type: "+print_type(pt));
- ne = AlgebraicOptimization.translate_all(ne);
- if (Config.trace)
- System.out.println("Translated expression:\n"+ne.pretty(0));
- Tree et = TypeInference.type_inference(ne);
- is_dataset = PlanGeneration.is_dataset_expr(ne);
- if (Config.trace)
- System.out.println("Physical plan type: "+print_type(et));
- repeat_variables = #[];
- ne = Simplification.simplify_all(ne);
- Tree plan = PlanGeneration.makePlan(ne);
- if (Config.bsp_mode) {
- BSPTranslator.reset();
- if (Config.trace)
- System.out.println("Physical plan:\n"+plan.pretty(0));
- plan = Materialization.materialize_terms(BSPTranslator.constructBSPplan(plan));
- if (Config.trace)
- System.out.println("BSP plan:\n"+plan.pretty(0));
- else {
- String splan = print_plan(plan,0,false);
- if (!splan.equals("") && !Config.quiet_execution)
- System.out.println("BSP plan:\n"+splan);
- }
- } else {
- if (Config.hadoop_mode)
- plan = PlanGeneration.physical_plan(plan);
- plan = Materialization.materialize_terms(AlgebraicOptimization.common_factoring(plan));
- if (Config.trace)
- System.out.println("Physical plan:\n"+plan.pretty(0));
- else {
- String splan = print_plan(plan,0,false);
- if (!splan.equals("") && !Config.quiet_execution)
- System.out.println("Physical plan:\n"+splan);
- }
- };
- if (Config.compile_functional_arguments)
- plan = Compiler.compile(plan);
- return plan;
- } catch (Error x) {
- if (Config.testing)
- throw new Error(x);
- if (!Config.trace && x.toString().endsWith("Type Error"))
- return null;
- if (x.getMessage() != null) // system error
- System.err.println("*** MRQL System Error at line "+Main.parser.line_pos()+": "+x);
- if (Config.trace)
- x.printStackTrace(System.err);
- return null;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Inv.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Inv.java b/src/main/java/core/Inv.java
deleted file mode 100644
index 3faa7fe..0000000
--- a/src/main/java/core/Inv.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-
-
-final public class Inv extends MRData {
- private MRData value;
-
- Inv ( MRData v ) { value = v; }
-
- public void materializeAll () { value.materializeAll(); };
-
- public MRData value () { return value; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.INV);
- value.write(out);
- }
-
- final public static Inv read ( DataInput in ) throws IOException {
- return new Inv(MRContainer.read(in));
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value.readFields(in);
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof Inv);
- return -value.compareTo(((Inv)x).value);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- int n = MRContainer.compare(x,xs,xl,y,ys,yl,size);
- size[0] += 1;
- return -n;
- }
-
- public boolean equals ( Object x ) {
- return x instanceof Inv && value.equals(((Inv)x).value);
- }
-
- public int hashCode () { return value.hashCode(); }
-
- public String toString () { return "inv("+value.toString()+")"; }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/JSON.cup
----------------------------------------------------------------------
diff --git a/src/main/java/core/JSON.cup b/src/main/java/core/JSON.cup
deleted file mode 100644
index c045437..0000000
--- a/src/main/java/core/JSON.cup
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java_cup.runtime.*;
-
-terminal TRUE, FALSE, NULL, COLON, COMMA, O_BEGIN, O_END, A_BEGIN, A_END;
-
-terminal String STRING;
-terminal Long INTEGER;
-terminal Double DOUBLE;
-
-non terminal MRData top, json, pair, value;
-non terminal Bag members, elements;
-
-precedence left O_BEGIN, O_END, A_BEGIN, A_END;
-precedence left COMMA;
-precedence left COLON;
-
-start with top;
-
-top ::= json:e {: RESULT = e; :}
- ;
-json ::= O_BEGIN O_END {: RESULT = new Union((byte)0,new Bag()); :}
- | O_BEGIN members:m O_END {: RESULT = new Union((byte)0,m); :}
- | A_BEGIN A_END {: RESULT = new Union((byte)1,new Bag()); :}
- | A_BEGIN elements:m A_END {: RESULT = new Union((byte)1,m); :}
- | O_BEGIN O_BEGIN INTEGER:n O_END O_END {: RESULT = new MR_variable((int)n.longValue()); :}
- ;
-members ::= pair:p {: RESULT = new Bag(p); :}
- | members:m COMMA pair:p {: RESULT = m.add_element(p); :}
- ;
-pair ::= STRING:s COLON value:v {: RESULT = (new Tuple(2)).set(0,new MR_string(s)).set(1,v); :}
- ;
-elements ::= value:v {: RESULT = new Bag(v); :}
- | elements:es COMMA value:v {: RESULT = es.add_element(v); :}
- ;
-value ::= json:e {: RESULT = e; :}
- | STRING:s {: RESULT = new Union((byte)2,new MR_string(s)); :}
- | INTEGER:n {: RESULT = new Union((byte)3,new MR_long(n.longValue())); :}
- | DOUBLE:n {: RESULT = new Union((byte)4,new MR_double(n.doubleValue())); :}
- | TRUE {: RESULT = new Union((byte)5,new MR_bool(true)); :}
- | FALSE {: RESULT = new Union((byte)5,new MR_bool(false)); :}
- | NULL {: RESULT = new Union((byte)6,new Tuple(0)); :}
- ;
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/JSON.lex
----------------------------------------------------------------------
diff --git a/src/main/java/core/JSON.lex b/src/main/java/core/JSON.lex
deleted file mode 100644
index c75d1bb..0000000
--- a/src/main/java/core/JSON.lex
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java_cup.runtime.Symbol;
-
-
-%%
-%class JSONLex
-%public
-%line
-%char
-%cup
-%eofval{
- return symbol(jsym.EOF);
-%eofval}
-%{
- public String text () { return yytext(); }
-
- public int line_pos () { return yyline+1; }
-
- public int char_pos () { return yychar; }
-
- public static Symbol symbol ( int s ) {
- return new Symbol(s);
- }
-
- public static Symbol symbol ( int s, Object o ) {
- return new Symbol(s,o);
- }
-
-%}
-
-INT = [+-]?[0-9]+
-DOUBLE = [+-]?[0-9]+([\.][0-9]+)?([eE][+-]?[0-9]+)?
-
-%%
-
-\"[^\"]*\" { return symbol(jsym.STRING,yytext().substring(1,yytext().length()-1)); }
-{INT} { return symbol(jsym.INTEGER,new Long(yytext())); }
-{DOUBLE} { return symbol(jsym.DOUBLE,new Double(yytext())); }
-true { return symbol(jsym.TRUE); }
-false { return symbol(jsym.FALSE); }
-null { return symbol(jsym.NULL); }
-\{ { return symbol(jsym.O_BEGIN); }
-\} { return symbol(jsym.O_END); }
-\[ { return symbol(jsym.A_BEGIN); }
-\] { return symbol(jsym.A_END); }
-\, { return symbol(jsym.COMMA); }
-\: { return symbol(jsym.COLON); }
-[ \t\f] { }
-[\r\n] { }
-. { throw new Error("Illegal character: "+yytext()); }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/JsonFormatParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/JsonFormatParser.java b/src/main/java/core/JsonFormatParser.java
deleted file mode 100644
index 77aa891..0000000
--- a/src/main/java/core/JsonFormatParser.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.StringReader;
-import java.nio.ByteBuffer;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.DataOutputBuffer;
-import java_cup.runtime.Symbol;
-
-
-/** The JSON parser */
-public class JsonFormatParser implements Parser {
- String[] tags; // split tags
- JsonSplitter splitter;
-
- public void initialize ( Trees args ) {
- try {
- if (args.length() > 0) {
- if (!(args.nth(0) instanceof Node)
- || !(((Node)args.nth(0)).name().equals("list")
- || ((Node)args.nth(0)).name().equals("bag")))
- throw new Error("Must provide a bag of synchronization property names to split the JSON source: "+args.nth(0));
- Trees ts = ((Node)args.nth(0)).children();
- if (ts.length() == 0)
- throw new Error("Expected at least one synchronization tagname in JSON source: "+ts);
- tags = new String[ts.length()];
- for ( int i = 0; i < tags.length; i++ )
- if (ts.nth(i) instanceof StringLeaf)
- tags[i] = ((StringLeaf)(ts.nth(i))).value();
- else throw new Error("Expected a synchronization tagname in JSON source: "+ts.nth(i));
- }
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public void open ( String file ) {
- try {
- splitter = new JsonSplitter(tags,file,new DataOutputBuffer());
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public void open ( FSDataInputStream fsin, long start, long end ) {
- try {
- splitter = new JsonSplitter(tags,fsin,start,end,new DataOutputBuffer());
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public Tree type () { return new VariableLeaf("JSON"); }
-
- public String slice () {
- if (splitter.hasNext()) {
- DataOutputBuffer b = splitter.next();
- return new String(b.getData(),0,b.getLength());
- } else return null;
- }
-
- public Bag parse ( String s ) {
- try {
- JSONLex scanner = new JSONLex(new StringReader(s));
- JSONParser parser = new JSONParser(scanner);
- Symbol sym = parser.parse();
- return new Bag((MRData)sym.value);
- } catch (Exception e) {
- System.err.println(e);
- return new Bag();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/JsonSplitter.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/JsonSplitter.java b/src/main/java/core/JsonSplitter.java
deleted file mode 100644
index 7017e72..0000000
--- a/src/main/java/core/JsonSplitter.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java_cup.runtime.Symbol;
-import java.util.Iterator;
-import java.io.*;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.io.DataOutputBuffer;
-
-
-/** Extract the JSON objects tagged by tags from a data split of the input stream (fsin) */
-final public class JsonSplitter implements Iterator<DataOutputBuffer> {
- boolean in_memory;
- FSDataInputStream fsin; // for HDFS processing
- InputStream in; // for in-memory processing
- JSONLex scanner;
- String[] tags;
- long start;
- long end;
- final DataOutputBuffer buffer;
-
- JsonSplitter ( String[] tags, FSDataInputStream fsin, long start, long end,
- DataOutputBuffer buffer ) {
- in_memory = false;
- this.tags = tags;
- this.fsin = fsin;
- this.end = end;
- this.buffer = buffer;
- try {
- fsin.seek(start);
- this.start = (start == 0) ? start : sync(start);
- fsin.seek(this.start);
- scanner = new JSONLex(fsin);
- } catch ( IOException e ) {
- System.err.println("*** Cannot parse the data split: "+fsin);
- }
- }
-
- JsonSplitter ( String[] tags, String file, DataOutputBuffer buffer ) {
- in_memory = true;
- try {
- in = new FileInputStream(file);
- } catch ( Exception e ) {
- throw new Error("Cannot open the file: "+file);
- };
- this.tags = tags;
- this.buffer = buffer;
- scanner = new JSONLex(in);
- }
-
- private long sync ( long start ) {
- try {
- long first_quote = -1;
- for ( long offset = 0; ; offset++ ) {
- char c = (char)fsin.read();
- if (c == '\"') {
- if (first_quote >= 0)
- if ((char)fsin.read() == ':')
- return start+first_quote;
- first_quote = offset;
- }
- }
- } catch (IOException ex) {
- return (long)0;
- }
- }
-
- public boolean hasNext () {
- try {
- if (in_memory || start+scanner.char_pos() < end)
- if (skip())
- return store();
- return false;
- } catch (Exception e) {
- System.err.println(e);
- return false;
- }
- }
-
- public DataOutputBuffer next () {
- return buffer;
- }
-
- public void remove () { }
-
- boolean is_start_tag ( String tagname ) {
- if (tags == null)
- return true;
- for (String tag: tags)
- if (tag.contentEquals(tagname))
- return true;
- return false;
- }
-
- /** skip until the beginning of a split element */
- boolean skip () throws IOException {
- while (true) {
- Symbol s = scanner.next_token();
- if (s.sym == jsym.EOF || (!in_memory && start+scanner.char_pos() >= end))
- return false;
- if (s.sym == jsym.STRING && is_start_tag((String)s.value)) {
- String tag = (String)s.value;
- if (scanner.next_token().sym == jsym.COLON) {
- buffer.reset();
- buffer.write('{');
- buffer.write('\"');
- for ( int i = 0; i < tag.length(); i++ )
- buffer.write(tag.charAt(i));
- buffer.write('\"');
- buffer.write(':');
- return true;
- }
- }
- }
- }
-
- /** store one split element into the buffer; may cross split boundaries */
- boolean store () throws IOException {
- int nest = 0;
- while (true) {
- Symbol s = scanner.next_token();
- if (s.sym == jsym.EOF)
- return false;
- if (s.sym == jsym.O_BEGIN || s.sym == jsym.A_BEGIN)
- nest++;
- else if (s.sym == jsym.O_END || s.sym == jsym.A_END)
- nest--;
- String text = scanner.text();
- for ( int i = 0; i < text.length(); i++ )
- buffer.write(text.charAt(i));
- if (nest == 0) {
- buffer.write('}');
- return true;
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Lambda.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Lambda.java b/src/main/java/core/Lambda.java
deleted file mode 100644
index d0cfd7a..0000000
--- a/src/main/java/core/Lambda.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** an anonymous function encapsulated as MRData */
-final public class Lambda extends MRData {
- private Function lambda;
-
- public Lambda ( Function f ) { lambda = f; }
-
- public void materializeAll () {};
-
- public Function lambda () { return lambda; }
-
- final public void write ( DataOutput out ) throws IOException {
- throw new Error("Functions are not serializable");
- }
-
- public void readFields ( DataInput in ) throws IOException {
- throw new Error("Functions are not serializable");
- }
-
- public int compareTo ( MRData x ) {
- throw new Error("Functions cannot be compared");
- }
-
- public boolean equals ( Object x ) {
- throw new Error("Functions cannot be compared");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/LineParser.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/LineParser.gen b/src/main/java/core/LineParser.gen
deleted file mode 100644
index fb1e068..0000000
--- a/src/main/java/core/LineParser.gen
+++ /dev/null
@@ -1,232 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.util.LineReader;
-
-
-/** A parser for line-oriented, character delimited text files (such as CVS) */
-final public class LineParser implements Parser {
- final static int maxLineLength = 1000;
- boolean in_memory;
- FSDataInputStream fsin; // for HDFS processing
- LineReader in;
- BufferedReader buffered_in; // for in-memory processing
- Text line;
- long start;
- long end;
- long pos;
- String delimiter;
- Tree type;
- byte[] types; // a vector of basic type ids (see MRContainer in MapReduceData)
- int type_length;
-
- static byte[] relational_record ( Tree tp ) {
- match tp {
- case record(...al):
- Trees attrs = #[];
- byte[] types = new byte[al.length()];
- for ( int i = 0; i < types.length; i++ )
- match al.nth(i) {
- case bind(`v,any):
- types[i] = -1;
- if (attrs.member(v))
- TypeInference.type_error(tp,"Duplicate record attribute name: "+v);
- attrs = attrs.append(v);
- case bind(`v,`t):
- if (!t.is_variable())
- fail;
- types[i] = MRContainer.type_code(t.toString());
- if (attrs.member(v))
- TypeInference.type_error(tp,"Duplicate record attribute name: "+v);
- attrs = attrs.append(v);
- if (!MRContainer.basic_type(types[i]))
- TypeInference.error("Expected a basic type for a relational attribute: "+t);
- case `t: TypeInference.error("Expected a basic type for a relational attribute: "
- +TypeInference.print_type(t));
- };
- return types;
- case tuple(...al):
- byte[] types = new byte[al.length()];
- for ( int i = 0; i < types.length; i++ )
- match al.nth(i) {
- case any:
- types[i] = -1;
- case `t:
- if (!t.is_variable())
- fail;
- types[i] = MRContainer.type_code(t.toString());
- if (!MRContainer.basic_type(types[i]))
- TypeInference.error("Expected a basic type for a relational attribute: "+t);
- case `t: TypeInference.error("Expected a basic type for a relational attribute: "
- +TypeInference.print_type(t));
- };
- return types;
- };
- TypeInference.error("Expected a relational record or a tuple type: "
- +TypeInference.print_type(tp));
- return null;
- }
-
- static Tree relational_record_type ( Tree tp ) {
- match tp {
- case record(...al):
- Trees ts = #[];
- for ( Tree a: al )
- match a {
- case bind(_,any): ;
- case `t: ts = ts.append(t);
- };
- return #<record(...ts)>;
- case tuple(...al):
- Trees ts = #[];
- for ( Tree a: al )
- if (!a.equals(#<any>))
- ts = ts.append(a);
- return #<tuple(...ts)>;
- };
- TypeInference.error("Expected a relational record type: "
- +TypeInference.print_type(tp));
- return null;
- }
-
- public Tree type () {
- return relational_record_type(type);
- }
-
- public void initialize ( Trees args ) {
- if (Config.hadoop_mode && Plan.conf == null)
- Plan.conf = Evaluator.evaluator.new_configuration();
- if (args.length() != 2)
- throw new Error("The line parser must have two arguments: "+args);
- if (!(args.nth(0) instanceof StringLeaf))
- throw new Error("Expected a delimiter: "+args.nth(0));
- delimiter = ((StringLeaf)args.nth(0)).value();
- if (delimiter.length() == 0)
- throw new Error("Expected a delimiter with at least one character: "+delimiter);
- type = ((Node)args.nth(1)).children().nth(0);
- types = relational_record(type);
- type_length = 0;
- for ( int i = 0; i < types.length; i++ )
- if (types[i] >= 0)
- type_length++;
- if (type_length < 1)
- TypeInference.error("A relational record type must have at least one component: "
- +TypeInference.print_type(type));
- }
-
- public void open ( String file ) {
- in_memory = true;
- try {
- buffered_in = new BufferedReader(new InputStreamReader(new FileInputStream(file)),
- 10000);
- } catch ( Exception e ) {
- throw new Error("Cannot open the file: "+file);
- }
- }
-
- public void open ( FSDataInputStream fsin, long fstart, long fend ) {
- in_memory = false;
- this.fsin = fsin;
- start = fstart;
- end = fend;
- line = new Text();
- try {
- if (start != 0) { // for all but the first data split, skip the first record
- --start;
- fsin.seek(start);
- in = new LineReader(fsin,Plan.conf);
- start += in.readLine(new Text(),0,(int) Math.min(Integer.MAX_VALUE,end-start));
- } else in = new LineReader(fsin,Plan.conf);
- pos = start;
- } catch ( IOException e ) {
- System.err.println("*** Cannot parse the data split: "+fsin);
- this.start = end;
- }
- }
-
- public String slice () {
- try {
- if (in_memory)
- return buffered_in.readLine();
- while (pos < end) {
- int newSize = in.readLine(line,maxLineLength,
- Math.max((int)Math.min(Integer.MAX_VALUE,end-pos),
- maxLineLength));
- if (newSize == 0)
- return null;
- pos += newSize;
- if (newSize < maxLineLength)
- return line.toString();
- };
- return null;
- } catch ( Exception e ) {
- System.err.println("*** Cannot slice the text: "+e);
- return "";
- }
- }
-
- private static MRData parse_value ( String text, byte type ) {
- switch (type) {
- case MRContainer.BYTE: return new MR_byte(Byte.parseByte(text));
- case MRContainer.SHORT: return new MR_short(Short.parseShort(text));
- case MRContainer.INT: return new MR_int(Integer.parseInt(text));
- case MRContainer.LONG: return new MR_long(Long.parseLong(text));
- case MRContainer.FLOAT: return new MR_float(Float.parseFloat(text));
- case MRContainer.DOUBLE: return new MR_double(Double.parseDouble(text));
- case MRContainer.CHAR: return new MR_char(text.charAt(0));
- case MRContainer.STRING: return new MR_string(text);
- };
- System.err.println("*** Cannot parse the type "+MRContainer.type_names[type]+" in '"+text+"'");
- return null;
- }
-
- public Bag parse ( String line ) {
- try {
- if (line == null)
- return new Bag();
- Tuple t = new Tuple(type_length);
- int loc = 0;
- int j = 0;
- for ( int i = 0; i < types.length; i++ ) {
- int k = line.indexOf(delimiter,loc);
- if (types[i] >= 0) {
- String s = (k > 0) ? line.substring(loc,k) : line.substring(loc);
- MRData v = parse_value(s,types[i]);
- if (v == null)
- return new Bag();
- t.set(j++,v);
- };
- loc = k+delimiter.length();
- if (k < 0 && i+1 < types.length) {
- System.err.println("*** Incomplete parsed text line: "+line);
- return new Bag();
- }
- };
- return new Bag(t);
- } catch ( Exception e ) {
- System.err.println("*** Cannot parse the text line: "+line);
- return new Bag();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MRContainer.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MRContainer.java b/src/main/java/core/MRContainer.java
deleted file mode 100644
index d72c452..0000000
--- a/src/main/java/core/MRContainer.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.io.WritableComparable;
-
-
-/** A container for MRData that implements read (the deserializer) */
-final public class MRContainer implements WritableComparable<MRContainer>, Serializable {
- transient MRData data;
-
- public final static byte BOOLEAN = 0, BYTE = 1, SHORT = 2, INT = 3, LONG = 4,
- FLOAT = 5, DOUBLE = 6, CHAR = 7, STRING = 8, PAIR = 9, TUPLE = 10, BAG = 11,
- LAZY_BAG = 12, END_OF_LAZY_BAG = 13, UNION = 14, INV = 15, LAMBDA = 16,
- VARIABLE = 17, TRIPLE = 18, NULL = 19, DATASET = 20, SYNC = 99, MORE_BSP_STEPS = 98;
-
- public final static byte[] type_codes
- = { BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, CHAR, STRING, NULL, PAIR, TRIPLE,
- TUPLE, BAG, LAZY_BAG, END_OF_LAZY_BAG, UNION, INV, LAMBDA, VARIABLE, SYNC };
-
- public final static String[] type_names
- = { "boolean", "byte", "short", "int", "long", "float", "double", "char", "string",
- "null", "pair", "triple", "tuple", "bag", "lazy_bag", "end_of_lazy_bag", "union",
- "inv", "lambda", "variable", "sync", "more_bsp_steps" };
-
- public static byte type_code ( String type_name ) {
- for ( byte i = 0; i < type_names.length; i ++ )
- if (type_names[i].equals(type_name))
- return type_codes[i];
- return -1;
- }
-
- public static boolean basic_type ( byte type_code ) {
- return type_code >= 0 && type_code <= 8;
- }
-
- MRContainer ( MRData d ) { data = d; }
-
- MRContainer () { data = null; }
-
- public final static MRData end_of_lazy_bag = new MR_EOLB();
-
- MRData data () { return data; }
-
- public void set ( MRData v ) { data = v; }
- final public void write ( DataOutput out ) throws IOException { data.write(out); }
- public void readFields ( DataInput in ) throws IOException { data = read(in); }
- public int compareTo ( MRContainer x ) { return data.compareTo(x.data); }
- public boolean equals ( Object x ) { return data.equals(x); }
- public int hashCode () { return data.hashCode(); }
- public String toString () { return data.toString(); }
-
- final public static MRData read ( DataInput in ) throws IOException {
- final byte tag = in.readByte();
- switch (tag) {
- case TUPLE: return Tuple.read(in);
- case NULL: return new Tuple(0);
- case PAIR: return Tuple.read2(in);
- case TRIPLE: return Tuple.read3(in);
- case BAG: return Bag.read(in);
- case LAZY_BAG: return Bag.lazy_read(in);
- case END_OF_LAZY_BAG: return end_of_lazy_bag;
- case UNION: return Union.read(in);
- case INV: return Inv.read(in);
- case BOOLEAN: return MR_bool.read(in);
- case BYTE: return MR_byte.read(in);
- case SHORT: return MR_short.read(in);
- case INT: return MR_int.read(in);
- case LONG: return MR_long.read(in);
- case FLOAT: return MR_float.read(in);
- case DOUBLE: return MR_double.read(in);
- case CHAR: return MR_char.read(in);
- case STRING: return MR_string.read(in);
- case SYNC: return new MR_sync();
- case MORE_BSP_STEPS: return new MR_more_bsp_steps();
- };
- throw new Error("Unrecognized MRQL type tag: "+tag);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- if (x[xs] != y[ys])
- return x[xs] - y[ys];
- switch (x[xs]) {
- case TUPLE: return Tuple.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case NULL: return 0;
- case PAIR: return Tuple.compare2(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case TRIPLE: return Tuple.compare3(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case BAG: return Bag.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case UNION: return Union.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case INV: return Inv.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case BOOLEAN: return MR_bool.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case BYTE: return MR_byte.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case SHORT: return MR_short.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case INT: return MR_int.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case LONG: return MR_long.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case FLOAT: return MR_float.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case DOUBLE: return MR_double.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case CHAR: return MR_char.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case STRING: return MR_string.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
- case SYNC: return 0;
- case MORE_BSP_STEPS: return 0;
- };
- throw new Error("Unrecognized MRQL type tag: "+x[xs]);
- }
-
- private void writeObject ( ObjectOutputStream out ) throws IOException {
- data.write(out);
- }
-
- private void readObject ( ObjectInputStream in ) throws IOException, ClassNotFoundException {
- data = read(in);
- }
-
- private void readObjectNoData () throws ObjectStreamException {}
-
- final static class MR_EOLB extends MRData {
- MR_EOLB () {}
-
- public void materializeAll () {};
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.END_OF_LAZY_BAG);
- }
-
- public void readFields ( DataInput in ) throws IOException {}
-
- public int compareTo ( MRData x ) { return 0; }
-
- public boolean equals ( Object x ) { return x instanceof MR_EOLB; }
-
- public int hashCode () { return 0; }
-
- public String toString () {
- return "end_of_lazy_bag";
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MRData.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MRData.java b/src/main/java/core/MRData.java
deleted file mode 100644
index 8c08269..0000000
--- a/src/main/java/core/MRData.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.hadoop.io.WritableComparable;
-import java.io.*;
-
-
-/** All MRQL data are encoded as MRData (similar to AVRO form) */
-public abstract class MRData implements WritableComparable<MRData>, Serializable {
- abstract public void materializeAll ();
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MRQL.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MRQL.java b/src/main/java/core/MRQL.java
deleted file mode 100644
index 7ba081e..0000000
--- a/src/main/java/core/MRQL.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-
-
-/** API for dynamic MRQL queries */
-final public class MRQL extends Interpreter {
-
- /** evaluate an MRQL query in a string
- * @param query a string that contains an MRQL query
- * @return the evaluation result
- */
- public static MRData query ( String query ) {
- evaluate("store tt := "+query+";");
- return variable_lookup("tt",global_env);
- }
-
- /** evaluate MRQL statments in a string
- * @param command a string that contains MRQL commands separated by ;
- */
- public static void evaluate ( String command ) {
- try {
- MRQLParser parser = new MRQLParser();
- parser.setScanner(new MRQLLex(new StringReader(command)));
- MRQLLex.reset();
- parser.parse();
- } catch (Exception x) {
- x.printStackTrace();
- throw new Error(x);
- }
- }
-
- /** clean up the MRQL workspace */
- public static void clean () {
- try {
- Plan.clean();
- } catch (IOException ex) {
- throw new Error("Failed to clean-up temporary files");
- }
- }
-}
[06/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Printer.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Printer.gen b/src/main/java/core/Printer.gen
deleted file mode 100644
index d0d80da..0000000
--- a/src/main/java/core/Printer.gen
+++ /dev/null
@@ -1,483 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.Iterator;
-import java.util.List;
-
-
-/** printers for types, expressions, plans, etc */
-public class Printer {
-
- public static String print_type ( Tree tp ) {
- match tp {
- case tuple(...tl):
- if (tl.is_empty())
- return "()";
- String s = "( "+print_type(tl.head());
- for ( Tree t: tl.tail() )
- s += ", "+print_type(t);
- return s+" )";
- case record(...tl):
- if (tl.is_empty())
- return "< >";
- String s = "< ";
- match tl.head() {
- case bind(`a,`t):
- s += a+": "+print_type(t);
- };
- for ( Tree t: tl.tail() )
- match t {
- case bind(`a,`at):
- s += ", "+a+": "+print_type(at);
- };
- return s+" >";
- case arrow(`itp,`otp):
- return print_type(itp)+" -> "+print_type(otp);
- case persistent(`t):
- return "!"+print_type(t);
- case `f():
- return f+"()";
- case persistent(`t):
- return "!"+print_type(t);
- case Bag(`etp):
- return "!bag("+print_type(etp)+")";
- case List(`etp):
- return "!list("+print_type(etp)+")";
- case `f(...tl):
- String s = f+"( "+print_type(tl.head());
- for ( Tree t: tl.tail() )
- s += ", "+print_type(t);
- return s+" )";
- };
- return tp.toString();
- }
-
- private static String print_query_list ( Trees el ) {
- if (el.length() == 0)
- return "";
- String s = " ";
- s += print_query(el.head());
- for ( Tree a: el.tail() )
- s += ", "+print_query(a);
- return s+" ";
- }
-
- public static String print_query ( Tree e ) {
- match e {
- case select(`opt_dist,`u,from(...bl),where(`c),groupby(...gs),orderby(...os)):
- String s = "select "+(opt_dist.equals(#<none>) ? "" : "distinct ");
- s += print_query(u)+" from ";
- match bl.head() {
- case bind(`p,`d):
- s += print_query(p)+" in "+print_query(d);
- };
- for ( Tree b: bl.tail() )
- match b {
- case bind(`p,`d):
- s += ", "+print_query(p)+" in "+print_query(d);
- };
- if (!c.equals(#<true>))
- s += " where "+print_query(c);
- match #<groupby(...gs)> {
- case groupby(`h,...gl):
- s += " group by ";
- match gl.head() {
- case bind(`gp,`gd):
- s += print_query(gp)+": "+print_query(gd);
- };
- for ( Tree g: gl.tail() )
- match g {
- case bind(`gp,`gd):
- s += ", "+print_query(gp)+": "+print_query(gd);
- };
- if (!h.equals(#<true>))
- s += " having "+print_query(h);
- };
- match #<orderby(...os)> {
- case orderby(`l,...ol):
- s += " order by "+print_query(ol.length() == 1 ? ol.head() : #<tuple(...ol)>);
- if (!l.equals(#<none>))
- s += " limit "+print_query(l);
- };
- return s;
- case tuple(...el):
- return "("+print_query_list(el)+")";
- case record(...el):
- String s = "< ";
- match el.head() {
- case bind(`v,`b):
- s += v+": "+print_query(b);
- };
- for ( Tree a: el.tail() )
- match a {
- case bind(`v,`b):
- s += ", "+v+": "+print_query(b);
- };
- return s+" >";
- case project(`a,`v):
- return (a.is_variable()) ? print_query(a)+"."+v : "("+print_query(a)+")."+v;
- case index(`a,`i):
- return (a.is_variable()) ? print_query(a)+"["+print_query(i)+"]"
- : "("+print_query(a)+")["+print_query(i)+"]";
- case nth(`x,`n):
- return (x.is_variable()) ? print_query(x)+"#"+print_query(n)
- : "("+print_query(x)+")#"+print_query(n);
- case call(`f,...el):
- return f+"("+print_query_list(el)+")";
- case list(...el):
- return "["+print_query_list(el)+"]";
- case bag(...el):
- return "{"+print_query_list(el)+"}";
- case `f(...el):
- return f+"("+print_query_list(el)+")";
- };
- return e.toString();
- }
-
- private static String print_XML ( final Union x ) {
- if (x.tag() == 1)
- return ((MR_string)x.value()).get();
- Tuple t = (Tuple)x.value();
- String s = "<"+((MR_string)t.get(0)).get();
- for ( MRData a: (Bag)t.get(1) ) {
- Tuple attr = (Tuple)a;
- s += " "+((MR_string)attr.first()).get()+"=\""
- +((MR_string)attr.second()).get()+"\"";
- };
- Bag c = (Bag)t.get(2);
- if (c.size() == 0)
- return s+"/>";
- s += ">";
- for ( MRData e: c )
- s += print_XML((Union)e);
- return s+"</"+((MR_string)t.get(0)).get()+">";
- }
-
- private static String print_JSON ( final Union x ) {
- switch (x.tag()) {
- case 0:
- String s = "{ ";
- for ( MRData e: (Bag)x.value() ) {
- Tuple t = (Tuple)e;
- s += t.get(0)+": "+print_JSON((Union)t.get(1))+", ";
- };
- return s.substring(0,s.length()-2)+" }";
- case 1:
- String q = "[ ";
- for ( MRData e: (Bag)x.value() )
- q += print_JSON((Union)e)+", ";
- return q.substring(0,q.length()-2)+" ]";
- };
- return ""+x.value();
- }
-
- /** An MRData printer based on type information */
- final static String print ( final MRData x, final Tree type ) {
- try {
- if (x instanceof Inv)
- return print(((Inv)x).value(),type);
- if (type.equals(#<XML>))
- return print_XML((Union)x);
- if (type.equals(#<JSON>))
- return print_JSON((Union)x);
- match TypeInference.expand(type) {
- case persistent(`tp):
- return print(x,tp);
- case Bag(`tp):
- if (x instanceof MR_dataset) {
- DataSet ds = ((MR_dataset)x).dataset();
- List<MRData> vals = ds.take(Config.max_bag_size_print);
- if (vals.size() == 0)
- return "{}";
- String s = "{ "+print(vals.get(0),tp);
- for ( int i = 1; i < vals.size(); i++ )
- s += ", "+print(vals.get(i),tp);
- if (vals.size() == Config.max_bag_size_print)
- return s+", ... }";
- else return s+" }";
- } else return print(x,#<bag(`tp)>);
- case List(`tp):
- if (x instanceof MR_dataset) {
- DataSet ds = ((MR_dataset)x).dataset();
- List<MRData> vals = ds.take(Config.max_bag_size_print);
- if (vals.size() == 0)
- return "[]";
- String s = "[ "+print(vals.get(0),tp);
- for ( int i = 1; i < vals.size(); i++ )
- s += ", "+print(vals.get(i),tp);
- if (vals.size() == Config.max_bag_size_print)
- return s+", ... ]";
- else return s+" ]";
- } else return print(x,#<list(`tp)>);
- case bag(`tp):
- Bag b = (Bag)x;
- Iterator<MRData> bi = b.iterator();
- if (!bi.hasNext())
- return "{}";
- String s = "{ "+print(bi.next(),tp);
- for ( long i = 1; bi.hasNext() && (Config.max_bag_size_print < 0
- || i < Config.max_bag_size_print); i++ )
- s += ", "+print(bi.next(),tp);
- if (bi.hasNext())
- return s+", ... }";
- else return s+" }";
- case list(`tp):
- Bag b = (Bag)x;
- Iterator<MRData> bi = b.iterator();
- if (!bi.hasNext())
- return "[]";
- String s = "[ "+print(bi.next(),tp);
- for ( long i = 1; bi.hasNext() && (Config.max_bag_size_print < 0
- || i < Config.max_bag_size_print); i++ )
- s += ", "+print(bi.next(),tp);
- if (bi.hasNext())
- return s+", ... ]";
- else return s+" ]";
- case tuple(...el):
- Tuple t = (Tuple)x;
- if (t.size() == 0)
- return "()";
- String s = "("+print(t.get((short)0),el.nth(0));
- for ( short i = 1; i < t.size(); i++ )
- s += ","+print(t.get(i),el.nth(i));
- return s+")";
- case record(...el):
- Tuple t = (Tuple)x;
- if (t.size() == 0)
- return "<>";
- String s = "< ";
- match el.nth(0) {
- case bind(`a,`tp):
- s += a+": "+print(t.get((short)0),tp);
- };
- for ( short i = 1; i < t.size(); i++ )
- match el.nth(i) {
- case bind(`a,`tp):
- s += ", "+a+": "+print(t.get(i),tp);
- };
- return s+" >";
- case union(...el):
- Union u = (Union)x;
- match el.nth(u.tag()) {
- case `c(tuple(...ts)):
- return c+print(u.value(),#<tuple(...ts)>);
- case `c(`tp):
- return c+"("+print(u.value(),tp)+")";
- }
- };
- return x.toString();
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
-
- private final static String tab ( int n ) {
- String s = "";
- for ( int i = 0; i < n; i++ )
- s += " ";
- return s;
- }
-
- /** print a physical plan
- * @param e the plan
- * @param n tab (# of spaces to put in the beginning of the line)
- * @param pv is this a variable bound to physical plan?
- * @return the printout
- */
- public final static String print_plan ( Tree e, int n, boolean pv ) {
- match e {
- case cMap(`f,`s):
- return "cMap:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
- case AggregateMap(`f,`a,`z,`s):
- return "AggregateMap:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
- case MapReduce(`m,`r,`s,_):
- return "MapReduce:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
- case MapCombineReduce(`m,`c,`r,`s,_):
- return "MapCombineReduce:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
- case MapAggregateReduce(`m,`r,`a,`z,`s,_):
- return "MapAggregateReduce:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
- case MapReduce2(`mx,`my,`r,`x,`y,_):
- return "MapReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,_):
- return "MapCombineReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case MapAggregateReduce2(`mx,`my,`r,`a,null,`x,`y,...):
- return "MapReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case MapAggregateReduce2(`mx,`my,`r,`a,`z,`x,`y,...):
- return "MapAggregateReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case MapJoin(`kx,`ky,`r,`x,`y):
- return "MapJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case MapAggregateJoin(`kx,`ky,`r,`a,null,`x,`y):
- return "MapJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case MapAggregateJoin(`kx,`ky,`r,`a,`z,`x,`y):
- return "MapAggregateJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case GroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,_):
- return "GroupByJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case CrossProduct(`mx,`my,`r,`x,`y):
- return "CrossProduct:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case CrossAggregateProduct(`mx,`my,`r,`a,null,`x,`y):
- return "CrossProduct:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case CrossAggregateProduct(`mx,`my,`r,`a,`z,`x,`y):
- return "CrossAggregateProduct:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case Aggregate(`a,`z,`s):
- return "Aggregate:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
- case BinarySource(`k,`file,_):
- return "Source (binary): "+file;
- case BinarySource(`file,_):
- return "Source (binary): "+file;
- case ParsedSource(`m,`parser,`file,...args):
- if (m instanceof LongLeaf)
- return "Source ("+parser+"): "+file;
- else fail
- case ParsedSource(`parser,`file,...args):
- return "Source ("+parser+"): "+file;
- case Generator(...):
- return "Generator";
- case Merge(`x,`y):
- return "Merge:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
- +tab(n+3)+"right: "+print_plan(y,n+10,true);
- case BSP(_,_,_,_,...ds):
- String ret = "BSP:\n";
- for ( Tree d: ds )
- ret += tab(n+3)+"input: "+print_plan(d,n+10,true);
- return ret;
- case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),...):
- String ret = "Loop ("+vs+"):\n"+tab(n+3)+"init: "
- +vs.head()+" = "+print_plan(ss.head(),n+vs.head().toString().length()+12,true)+"\n";
- ss = ss.tail();
- for ( Trees s = vs.tail(); !s.is_empty() && !ss.is_empty(); s = s.tail(), ss = ss.tail() )
- ret += tab(n+9)+s.head()+" = "+print_plan(ss.head(),n+s.head().toString().length()+12,true)+"\n";
- ret += tab(n+3)+"step: "
- +vs.head()+" = "+print_plan(bs.head(),n+vs.head().toString().length()+12,true)+"\n";
- bs = bs.tail();
- for ( Trees s = vs.tail(); !s.is_empty() && !bs.is_empty(); s = s.tail(), bs = bs.tail() )
- ret += tab(n+9)+s.head()+" = "+print_plan(bs.head(),n+s.head().toString().length()+12,true)+"\n";
- return ret;
- case `f(lambda(`v,`b),`s,...):
- if (! #[Repeat,repeat,Closure,closure].member(#<`f>))
- fail;
- return f+" ("+v+"):\n"+tab(n+3)+"init: "+print_plan(s,n+9,true)+"\n"
- +tab(n+3)+"step: "+print_plan(b,n+9,true);
- case Let(`v,`u,`body):
- return "let "+v+" = "+print_plan(u,n+10+v.toString().length(),pv)+"\n"
- +tab(n)+print_plan(body,n,pv);
- case If(_,`x1,If(_,`x2,If(_,`x3,`x4))):
- return "Choice 1: "+print_plan(x1,n+10,pv)+"\n"
- +tab(n)+"Choice 2: "+print_plan(x2,n+10,pv)+"\n"
- +tab(n)+"Choice 3: "+print_plan(x3,n+10,pv)+"\n"
- +tab(n)+"Choice 4: "+print_plan(x4,n+10,pv);
- case If(_,`x1,If(_,`x2,`x3)):
- return "Choice 1: "+print_plan(x1,n+10,pv)+"\n"
- +tab(n)+"Choice 2: "+print_plan(x2,n+10,pv)+"\n"
- +tab(n)+"Choice 3: "+print_plan(x3,n+10,pv);
- case If(`c,`x,`y):
- return "Choice 1: "+print_plan(x,n+10,pv)+"\n"
- +tab(n)+"Choice 2: "+print_plan(y,n+10,pv);
- case `f(...as):
- String s = "";
- for (Tree a: as) {
- String ps = print_plan(a,n,pv);
- if (!ps.equals("") && !a.is_variable())
- s += ps+(ps.endsWith("\n")?"":"\n");
- };
- return s;
- };
- if (pv && e.is_variable())
- return e.toString();
- return "";
- }
-
- /** given an MRData value, construct an expression that builds this data
- * @param x the MRData
- * @param type the type of x
- * @return an expression that constructs x
- */
- public final static Tree reify ( final MRData x, Tree type ) {
- if (x instanceof MR_variable)
- return new VariableLeaf("t_"+((MR_variable)x).var_num);
- type = TypeInference.expand(type);
- match type {
- case `T(`tp):
- if (!Translator.is_collection(T))
- fail;
- Bag b = (Bag)x;
- Trees as = #[];
- for ( MRData e: b)
- as = as.append(reify(e,tp));
- return #<`T(...as)>;
- case tuple(...el):
- Tuple t = (Tuple)x;
- Trees as = #[];
- for ( short i = 0; i < t.size(); i++ )
- as = as.append(reify(t.get(i),el.nth(i)));
- return #<tuple(...as)>;
- case record(...el):
- Tuple t = (Tuple)x;
- Trees as = #[];
- for ( short i = 0; i < t.size(); i++ )
- match el.nth(i) {
- case bind(`a,`tp):
- as = as.append(#<bind(`a,`(reify(t.get(i),tp)))>);
- };
- return #<record(...as)>;
- case union(...el):
- Union u = (Union)x;
- match el.nth(u.tag()) {
- case `c(tuple(...ts)):
- return #<call(`c,`(reify(u.value(),#<tuple(...ts)>)))>;
- case `c(`tp):
- return #<call(`c,`(reify(u.value(),tp)))>;
- };
- case string:
- String[] s = ((MR_string)x).get().split("\\x7B\\x7B");
- if (s.length == 1)
- return new StringLeaf(s[0]);
- Trees as = s[0].length() == 0 ? #[] : #[].append(new StringLeaf(s[0]));
- for ( int i = 1; i < s.length; i++ ) {
- String[] d = s[i].split("\\x7D\\x7D",2);
- if (d.length < 2)
- throw new Error("");
- as = as.append(new VariableLeaf("t_"+Integer.parseInt(d[0])));
- if (d[1].length() > 0)
- as = as.append(new StringLeaf(d[1]));
- };
- Tree res = as.reverse().head();
- for ( Tree a: as.reverse().tail() )
- res = #<call(plus,`a,`res)>;
- return res;
- case short: return #<typed(`(((MR_short)x).get()),`type)>;
- case int: return #<typed(`(((MR_int)x).get()),`type)>;
- case long: return #<typed(`((int)((MR_long)x).get()),`type)>;
- case float: return #<typed(`(((MR_float)x).get()),`type)>;
- case double: return #<typed(`((float)(((MR_double)x).get())),`type)>;
- };
- throw new Error("wrong type: "+type);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/QueryPlan.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/QueryPlan.gen b/src/main/java/core/QueryPlan.gen
deleted file mode 100644
index d9b907f..0000000
--- a/src/main/java/core/QueryPlan.gen
+++ /dev/null
@@ -1,999 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.*;
-
-
-/** Optimize a query plan by constructing a query graph and use a greedy
- * graph reduction algorithm to construct the query plan.
- * More details at: http://lambda.uta.edu/dood97.pdf
- */
-final public class QueryPlan {
- private static Hashtable<String,Tree> repeat_plans;
-
-private final static class SingleQueryPlan {
- static Tree[][] predicate; // the join predicate between two nodes
- static double[][] selectivity; // the selectivity of the join predicate
- static Tree[] plan; // the node plan
- static String[] var; // variable name
- static BitSet[] variables; // node variables
- static BitSet[] depends; // node dependencies
- static int[] depth; // min nesting level of the node
- static Tree[] pattern; // the pattern tree associated with the node
- static double[] size; // node cardinality
- static Tree[] filter; // the filter of a leaf dataset
- static Tree header; // the header of the root operator
- static Trees query_variables;// all query variables
- static SymbolTable header_binds; // the query header variables
- static HashMap<String,Integer> depths; // variable depths
- static boolean no_grouping; // true if we don't nest the operation results
-
- /** generate a fresh variable */
- static Tree new_var () { return Translator.new_var(); }
-
- /** true if the query domain is a collection retrieved from a data source */
- static boolean persistent_domain ( Tree e, Trees vars ) {
- if (contains_variables(e,vars)) // dependent to a persistent collection
- return true;
- match TypeInference.type_inference2(e) {
- case `T(_):
- if (Translator.is_persistent_collection(T)) // persistent collection
- return true;
- };
- return false;
- }
-
- /** true if the query domain is a collection retrieved from a data source */
- static boolean persistent_domain ( Tree e ) {
- return persistent_domain(e,query_variables);
- }
-
- /** the query bindings at any nesting level */
- static Trees all_binds ( Tree e, Trees vars ) {
- match e {
- case select(`u,from(...bl),where(`p)):
- Trees nl = #[];
- Trees vs = vars;
- for ( Tree b: bl )
- match b {
- case bind(`v,`d):
- nl = nl.append(all_binds(b,vs));
- if (persistent_domain(d,vs)) {
- vs = vs.append(v);
- nl = nl.append(b);
- }
- };
- return nl.append(all_binds(p,vs)).append(all_binds(u,vs));
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(all_binds(a,vars));
- return bl;
- };
- return #[];
- }
-
- static int var_index ( String name ) {
- for ( int i = 0; i < var.length; i++ )
- if (name.equals(var[i]))
- return i;
- return -1;
- }
-
- static void find_dependencies ( int i, Tree e ) {
- match e {
- case `f(...al):
- for (Tree a: al)
- find_dependencies(i,a);
- case `v:
- if (!v.is_variable())
- fail;
- String nm = ((VariableLeaf)v).value();
- int j = var_index(nm);
- if (j >= 0)
- depends[i].set(j);
- }
- }
-
- static int find_var ( Tree e ) {
- match e {
- case `f(...al):
- int i = -1;
- for (Tree a: al) {
- int j = find_var(a);
- if (j == -2)
- return j;
- else if (j >= 0)
- if (i >= 0 && i != j)
- i = -2;
- else i = j;
- };
- return i;
- case `v:
- if (!v.is_variable())
- fail;
- String nm = ((VariableLeaf)v).value();
- return var_index(nm);
- };
- return -1;
- }
-
- static boolean has_select ( Tree e ) {
- match e {
- case select(...):
- return true;
- case `f(...al):
- for (Tree a: al)
- if (has_select(a))
- return true;
- };
- return false;
- }
-
- static boolean contains_variables ( Tree e, Trees vars ) {
- match e {
- case `f(...al):
- for (Tree a: al)
- if (contains_variables(a,vars))
- return true;
- case _:
- if (vars.member(e))
- return true;
- };
- return false;
- }
-
- static Trees union ( Trees xs, Trees ys ) {
- Trees s = xs;
- for (Tree y: ys)
- if (!s.member(y))
- s = s.append(y);
- return s;
- }
-
- static Trees difference ( Trees xs, Trees ys ) {
- Trees s = #[];
- for (Tree x: xs)
- if (!ys.member(x))
- s = s.append(x);
- return s;
- }
-
- static Tree prime_expr ( Trees vars, Tree e ) {
- match e {
- case lambda(`p,`b):
- return #<lambda(`p,`(prime_expr(difference(vars,pattern_variables(p)),b)))>;
- case `f(...al):
- Trees s = #[];
- for (Tree a: al)
- s = s.append(prime_expr(vars,a));
- return #<`f(...s)>;
- case `v:
- if (v.is_variable())
- if (vars.member(v))
- return new VariableLeaf(((VariableLeaf)v).value()+"'");
- };
- return e;
- }
-
- static Trees pattern_variables ( Tree pat ) {
- match pat {
- case `f(...al):
- Trees s = #[];
- for (Tree a: al)
- s = s.append(pattern_variables(a));
- return s;
- case `v:
- if (v.is_variable())
- return #[`v];
- };
- return #[];
- }
-
- static Tree prime ( Tree pat, Tree e ) {
- return prime_expr(pattern_variables(pat),e);
- }
-
- static Tree subst_expr ( Tree var, Tree value, Tree e ) {
- match e {
- case lambda(`p,`b):
- if (pattern_variables(p).member(var))
- return e;
- else return #<lambda(`p,`(subst_expr(var,value,b)))>;
- case `f(...al):
- Trees s = #[];
- for (Tree a: al)
- s = s.append(subst_expr(var,value,a));
- return #<`f(...s)>;
- case `v:
- if (v.is_variable())
- if (v.equals(var))
- return value;
- };
- return e;
- }
-
- static Tree and ( Tree x, Tree y ) {
- if (x.equals(#<true>))
- return y;
- else if (y.equals(#<true>))
- return x;
- else return #<call(and,`x,`y)>;
- }
-
- static Tree find_predicates ( Tree e, Trees exclude_variables ) {
- match e {
- case call(and,`x,`y):
- return and(find_predicates(x,exclude_variables),
- find_predicates(y,exclude_variables));
- case call(eq,`x,`y):
- if (contains_variables(x,exclude_variables)
- || contains_variables(y,exclude_variables))
- fail;
- int i = find_var(x);
- int j = find_var(y);
- if (i >= 0 && j >= 0 && i != j) {
- predicate[i][j] = predicate[j][i]
- = (predicate[i][j].equals(#<true>)) ? e : and(e,predicate[i][j]);
- selectivity[i][j] = selectivity[j][i] = 0.01;
- return #<true>;
- } else if (i >= 0 && j == -1) {
- filter[i] = (filter[i].equals(#<true>)) ? e : and(e,filter[i]);
- plan[i] = #<cmap(lambda(`(var[i]),if(`e,bag(`(var[i])),bag())),`(plan[i]))>;
- return #<true>;
- } else if (j >= 0 && i == -1) {
- filter[j] = (filter[j].equals(#<true>)) ? e : and(e,filter[j]);
- plan[j] = #<cmap(lambda(`(var[j]),if(`e,bag(`(var[j])),bag())),`(plan[j]))>;
- return #<true>;
- }
- case call(`f,`x,`y):
- if (! #[ne,gt,geq,lt,leq].member(f))
- fail;
- if (has_select(x) || has_select(y))
- fail;
- if (contains_variables(x,exclude_variables)
- || contains_variables(y,exclude_variables))
- fail;
- int i = find_var(x);
- int j = find_var(y);
- if (i >= 0 && j < 0) {
- filter[i] = (filter[i].equals(#<true>)) ? e : and(e,filter[i]);
- plan[i] = #<cmap(lambda(`(var[i]),if(`e,bag(`(var[i])),bag())),`(plan[i]))>;
- return #<true>;
- } else if (i < 0 && j >= 0) {
- filter[j] = (filter[j].equals(#<true>)) ? e : and(e,filter[j]);
- plan[j] = #<cmap(lambda(`(var[j]),if(`e,bag(`(var[j])),bag())),`(plan[j]))>;
- return #<true>;
- }
- };
- return e;
- }
-
- static String tuple_var ( Tree x ) {
- match x {
- case cmap(_,`z): return tuple_var(z);
- };
- String s = x.toString();
- if (s.endsWith("'"))
- return s.substring(0,s.length()-1);
- return s;
- }
-
- /** reorder the tuple components in s based on the expected pattern variables in vars */
- static Tree tuple ( Trees s, Trees vars ) {
- if (s.length() != vars.length())
- throw new Error("Wrong pattern: "+s+" "+vars);
- if (s.length() == 1)
- return s.head();
- Tree[] v = new Tree[s.length()];
- for (Tree x: s) {
- int i = 0;
- for (Tree y: vars) {
- if (tuple_var(x).equals(y.toString()))
- v[i] = x;
- i++;
- };
- };
- Trees rs = #[];
- for ( int i = v.length-1; i >= 0 ; i-- )
- if (v[i] == null)
- throw new Error("Wrong pattern: "+s+" "+vars);
- else rs = rs.cons(v[i]);
- return #<tuple(...rs)>;
- }
-
- static class Header {
- public Tree header;
- public Trees pattern;
- Header ( Tree h, Trees p ) { header = h; pattern = p; }
- public String toString () { return header+" "+pattern; }
- }
-
- static Header build_graph ( Tree e, int level ) {
- match e {
- case select(`u,from(...bl),where(`p)):
- Trees nl = #[];
- Tree nv = new_var();
- Trees rest = #[];
- Trees exclude_variables = #[];
- depths.put(nv.toString(),new Integer(level));
- for (Tree b: bl)
- match b {
- case bind(`v,`d):
- if (!persistent_domain(d)) {
- exclude_variables = exclude_variables.append(v);
- rest = rest.append(b);
- continue;
- };
- String name = ((VariableLeaf)v).value();
- int i = var_index(name);
- Header nd = build_graph(d,level+1);
- plan[i] = nd.header;
- depth[i] = level;
- query_variables = query_variables.append(v);
- depths.put(name,new Integer(level+1));
- pattern[i] = #<`(nv.toString())(`v)>;
- find_dependencies(i,d);
- nl = nl.append(v);
- };
- if (nl.equals(#[]))
- return new Header(e,#[]);
- query_variables = query_variables.append(nv);
- Header nu = build_graph(u,level+1);
- Tree npred = find_predicates(p,exclude_variables);
- Header np = build_graph(npred,level+1);
- for (Tree b: nu.pattern)
- match b {
- case bind(`v,_): nl = nl.append(v);
- };
- for (Tree b: np.pattern)
- match b {
- case bind(`v,_): nl = nl.append(v);
- };
- Tree t = tuple(nl,nl);
- header_binds.insert(nv.toString(),t);
- return new Header(#<select(`(nu.header),from(bind(`t,`nv),...rest),where(`(np.header)))>,
- #[bind(`nv,`t)]);
- case `f(...al):
- Trees bl = #[];
- Trees nl = #[];
- for (Tree a: al) {
- Header n = build_graph(a,level);
- bl = bl.append(n.header);
- nl = nl.append(n.pattern);
- };
- return new Header(#<`f(...bl)>,nl);
- };
- return new Header(e,#[]);
- }
-
- static void dump ( int n ) {
- System.out.println("Query graph nodes:");
- for ( int i = 0; i < n; i++ ) {
- System.out.print(""+i+") "+variables(i)+" depth="+depth[i]+" pattern="+pattern[i]
- +" plan="+plan[i]+" size="+size[i]+" depends=(");
- for ( int j = 0; j < n; j++ )
- if (depends[i].get(j))
- System.out.print(""+j+" ");
- System.out.println(") "+filter[i]);
- };
- System.out.println("Query graph edges (predicates):");
- for ( int i = 0; i < n; i++ )
- for ( int j = 0; j < i; j++ )
- if (!predicate[i][j].equals(#<true>))
- System.out.println(""+i+" "+j+") "+predicate[i][j]);
- System.out.println("----------------------");
- }
-
- static Trees variables ( BitSet bs ) {
- Trees bl = #[];
- for ( int j = bs.nextSetBit(0); j >= 0; j = bs.nextSetBit(j+1)) {
- bl = bl.append(#<`(var[j])>);
- };
- return bl;
- }
-
- static Trees variables ( int i ) {
- return variables(variables[i]);
- }
-
- static Tree make_key ( Tree pred, BitSet vars ) {
- match pred {
- case call(and,`x,`y):
- return #<tuple(`(make_key(x,vars)),`(make_key(y,vars)))>;
- case call(eq,`x,`y):
- int i = find_var(x);
- if (i >= 0 && vars.get(i))
- return x;
- else return y;
- };
- return pred;
- }
-
- static boolean eq_patterns ( Trees xs, Tree y ) {
- Trees ys = (y.is_node()) ? ((Node)y).children() : #[`y];
- if (xs.length() != ys.length())
- return false;
- for (Tree x: xs)
- if (!ys.member(x))
- return false;
- return true;
- }
-
- static String pattern_head ( Tree x ) {
- String s = "";
- match x {
- case `g(...): return g;
- };
- return x.toString();
- }
-
- static Tree pattern_head ( Tree x, boolean prime ) {
- String s = pattern_head(x);
- return #<`(prime ? s+"'" : s)>;
- }
-
- static Trees pattern_children ( Tree x ) {
- match x {
- case _(...r): return r;
- };
- throw new Error("pattern is not a set: "+x);
- }
-
- static Tree pattern ( Tree p, boolean prime ) {
- Trees s = #[];
- match p {
- case `f(...r):
- for (Tree x: r)
- s = s.append(pattern_head(x,prime));
- case _: s = s.append(pattern_head(p,prime));
- };
- return (s.length() == 1) ? s.head() : #<tuple(...s)>;
- }
-
- static Trees pattern_children_variables ( Tree p ) {
- Trees s = #[];
- match p {
- case `f(...r):
- for (Tree x: r)
- s = s.append(pattern_head(x,false));
- case _: s = s.append(pattern_head(p,false));
- };
- return s;
- }
-
- static boolean contains ( Tree pattern, String var ) {
- match pattern {
- case `f(...s):
- if (f.equals(var))
- return true;
- for (Tree x: s)
- if (contains(x,var))
- return true;
- };
- return pattern.equals(#<`var>);
- }
-
- static Trees merge_patterns ( Tree x, Trees r ) {
- Trees s = #[];
- for (Tree y: r)
- if (pattern_overlap(x,y))
- s = s.append(merge_patterns(x,y));
- else s = s.append(y);
- if (!pattern_overlap(x,r))
- s = s.append(x);
- return s;
- }
-
- static Trees merge_patterns ( Trees r1, Trees r2 ) {
- Trees s = #[];
- for (Tree x: r1)
- if (pattern_overlap(x,r2))
- s = s.append(merge_patterns(x,r2));
- else s = s.append(x);
- for (Tree y: r2)
- if (!pattern_overlap(y,r1))
- s = s.append(y);
- return s;
- }
-
- static Tree merge_patterns ( Tree p1, Tree p2 ) {
- match p1 {
- case `f1(...r1):
- match p2 {
- case `f2(...r2):
- if (no_grouping || depth(f1) == depth(f2))
- return #<`f1(...(merge_patterns(r1,r2)))>;
- if (depth(f1) < depth(f2))
- return #<`f1(...(merge_patterns(p2,r1)))>;
- if (depth(f1) > depth(f2))
- return #<`f2(...(merge_patterns(p1,r2)))>;
- case _: return #<`f1(...(merge_patterns(p2,r1)))>;
- };
- case_ :
- match p2 {
- case `f2(...r2):
- return #<`f2(...(merge_patterns(p1,r2)))>;
- }
- };
- throw new Error("Cannot merge the pattern "+p1+" with "+p2);
- }
-
- static boolean pattern_overlap ( Tree x, Trees r ) {
- for (Tree y: r)
- if (pattern_overlap(x,y))
- return true;
- return false;
- }
-
- static boolean pattern_overlap ( Tree x, Tree y ) {
- match x {
- case `f1(...r1):
- match y {
- case `f2(...r2):
- if (f1.equals(f2)
- || contains(header_binds.lookup(f1),f2)
- || contains(header_binds.lookup(f2),f1))
- return true;
- case _: return contains(header_binds.lookup(f1),y.toString());
- };
- };
- return x.equals(y);
- }
-
- static Trees join_body ( Tree x, Trees r, Tree pred ) {
- Trees s = #[];
- if (!pattern_overlap(x,r))
- s = s.append((pred.equals(#<true>))
- ? pattern_head(x,false)
- : #<cmap(lambda(`(pattern(x,false)),if(`pred,bag(`(pattern(x,false))),bag())),
- `(pattern_head(x,false)))>);
- for (Tree y: r)
- if (pattern_overlap(x,y))
- s = s.append(join_body(x,y,#<true>));
- else s = s.append(pattern_head(y,true));
- return s;
- }
-
- static Trees join_body ( Trees r, Tree y, Tree pred ) {
- Trees s = #[];
- for (Tree x: r)
- if (pattern_overlap(x,y))
- s = s.append(join_body(x,y,#<true>));
- else s = s.append(pattern_head(x,false));
- if (!pattern_overlap(y,r))
- s = s.append((pred.equals(#<true>))
- ? pattern_head(y,true)
- : #<cmap(lambda(`(pattern(y,true)),if(`pred,bag(`(pattern(y,true))),bag())),
- `(pattern_head(y,true)))>);
- return s;
- }
-
- static Trees join_body ( Trees r1, Trees r2 ) {
- Trees s = #[];
- for (Tree x: r1)
- if (pattern_overlap(x,r2))
- s = s.append(join_body(x,r2,#<true>));
- else s = s.append(pattern_head(x,false));
- for (Tree y: r2)
- if (!pattern_overlap(y,r1))
- s = s.append(pattern_head(y,true));
- return s;
- }
-
- static int depth ( String n ) {
- return depths.get(n).intValue();
- }
-
- static Tree join_body ( Tree p1, Tree p2, Tree pred ) {
- Tree pat1 = pattern(p1,false);
- Tree pat2 = pattern(p2,true);
- Trees vars = pattern_children_variables(merge_patterns(p1,p2));
- match p1 {
- case `f1(...r1):
- match p2 {
- case `f2(...r2):
- if (no_grouping || depth(f1) == depth(f2)) {
- Tree t = tuple(join_body(r1,r2),vars);
- Tree body = (pred.equals(#<true>)) ? #<bag(`t)> : #<if(`pred,bag(`t),bag())>;
- return #<cmap(lambda(`pat1,cmap(lambda(`pat2,`body),`(f2+"'"))),
- `f1)>;
- } else if (depth(f1) < depth(f2)) {
- Tree t = tuple(join_body(r1,p2,pred),vars);
- return #<cmap(lambda(`pat1,bag(`t)),`f1)>;
- } else if (depth(f1) > depth(f2)) {
- Tree t = tuple(join_body(p1,r2,pred),vars);
- return #<cmap(lambda(`pat2,bag(`t)),`(f2+"'"))>; // 3/12/11: changed from `f2
- }
- }
- };
- throw new Error("wrong join: "+p1+" "+p2);
- }
-
- static Tree make_join ( int i, int j ) {
- Tree pi = pattern(pattern[i],false);
- Tree pj = pattern(pattern[j],false);
- Tree keyi = make_key(predicate[i][j],variables[i]);
- Tree keyj = make_key(predicate[i][j],variables[j]);
- Tree left = pattern_head(pattern[i],false);
- Tree right = pattern_head(pattern[j],true);
- Tree body = join_body(pattern[i],pattern[j],#<true>);
- if (Config.trace)
- System.out.print("join "+pattern[i]+" with "+pattern[j]);
- pattern[i] = merge_patterns(pattern[i],pattern[j]);
- if (Config.trace)
- System.out.println(" to get "+pattern[i]+" with body "+body);
- return #<join(lambda(`pi,`keyi),
- lambda(`pj,`keyj),
- lambda(tuple(`left,`right),`body),
- `(plan[i]),
- `(plan[j]))>;
- }
-
- private static Tree top_pattern_variables ( Tree pat ) {
- match pat {
- case _(...ts):
- Trees ps = #[];
- for ( Tree t: ts )
- match t {
- case `f(...): ps = ps.append(#<`f>);
- case _: ps = ps.append(t);
- };
- if (ps.length() > 1)
- return #<tuple(...ps)>;
- else return ps.head();
- };
- return pat;
- }
-
- static Tree make_unnest ( int i, int j ) {
- Tree body = null;
- if (Config.trace)
- System.out.print("unnest "+pattern[i]+" -> "+pattern[j]);
- if (!no_grouping && depth[i] < depth[j]) {
- // Changed 6/13/13: must rearrange binding variables in nested queries based on join order
- //body = subst_expr(pattern_head(pattern[j],false),plan[j],plan[i]);
- body = subst_header(pattern_head(pattern[j],false),top_pattern_variables(pattern[j]),plan[j],plan[i]);
- // new pattern[i] is the old pattern[i]
- } else {
- body = join_body(pattern[j],pattern[i],predicate[i][j]);
- body = prime(pattern[i],body);
- body = subst_expr(pattern_head(pattern[j],false),plan[j],
- subst_expr(pattern_head(pattern[i],true),plan[i],body));
- pattern[i] = merge_patterns(pattern[j],pattern[i]);
- };
- if (Config.trace)
- System.out.println(" to get "+pattern[i]+" with body "+body);
- return body;
- }
-
- static Tree make_map_join ( int i, int j ) {
- Tree body = join_body(pattern[i],pattern[j],predicate[i][j]);
- Tree left = pattern_head(pattern[i],false);
- Tree right = pattern_head(pattern[j],true);
- match body {
- case cmap(lambda(`x,cmap(lambda(`y,`b),`xx)),`yy):
- if (!xx.equals(right) || !yy.equals(left))
- fail;
- Tree nb = Meta.subst_expr(x,left,Meta.subst_expr(y,right,b));
- body = #<crossProduct(lambda(x,bag(x)),
- lambda(x,bag(x)),
- lambda(tuple(`left,`right),`nb),
- `(plan[i]),
- `(plan[j]))>;
- case cmap(lambda(`x,`b),`xx):
- if (!xx.equals(left))
- fail;
- body = Meta.subst_expr(x,xx,b);
- body = #<groupBy(crossProduct(lambda(x,bag(x)),
- lambda(x,bag(x)),
- lambda(tuple(`left,`right),`body),
- `(plan[i]),
- `(plan[j])))>;
- case _:
- body = prime(pattern[j],body);
- body = subst_expr(pattern_head(pattern[j],true),plan[j],
- subst_expr(pattern_head(pattern[i],false),plan[i],body));
- };
- if (Config.trace)
- System.out.print("cross product "+pattern[i]+" with "+pattern[j]);
- pattern[i] = merge_patterns(pattern[i],pattern[j]);
- if (Config.trace)
- System.out.println(" to get "+pattern[i]+" with body "+body);
- return body;
- }
-
- static Tree make_plan ( int i, int j ) {
- if (depends[i].get(j))
- return make_unnest(i,j);
- else if (predicate[i][j].equals(#<true>))
- return make_map_join(i,j);
- else return make_join(i,j);
- }
-
- /** node i should not have any join predicate with a node other than j */
- static boolean no_neighbors ( int i, int j, int n ) {
- for (int k = 0; k < n; k++)
- if (k != i && k != j && !predicate[i][k].equals(#<true>))
- return false;
- return true;
- }
-
- static boolean eligible ( int i, int j, int n ) {
- if (!depends[j].isEmpty()) // j must not have any dependency
- return false;
- else if (depends[i].isEmpty() // a join between i and j (neither i nor j have any dependency)
- || (depends[i].nextSetBit(0) == j
- && depends[i].nextSetBit(j+1) < 0)) { // i depends only on j
- if (no_grouping)
- return true;
- else if (depth[i] == depth[j])
- return true;
- else if (depth[i] < depth[j])
- return no_neighbors(j,i,n);
- else return no_neighbors(i,j,n);
- };
- return false;
- }
-
- static Tree subst_header ( Tree var, Tree pat, Tree plan, Tree header ) {
- match header {
- case bind(`p,`w):
- if (w.equals(var))
- return #<bind(`pat,`plan)>;
- else fail
- case `f(...al):
- Trees bl = #[];
- for (Tree a: al)
- bl = bl.append(subst_header(var,pat,plan,a));
- return #<`f(...bl)>;
- };
- return header;
- }
-
- static Tree ordered_tuple ( Trees xs ) {
- if (xs.length() == 1)
- return xs.head();
- Trees res = #[];
- for ( Tree v: query_variables )
- if (xs.member(v))
- res = res.append(v);
- return #<tuple(...res)>;
- }
-
- /** group-by the plan so that the flat results in xs are grouped at their proper level */
- static Tree final_groupBy ( int level, Trees xs, Tree plan ) {
- Trees rest = #[];
- Trees group_by_vars = #[];
- Trees pvars = #[];
- for ( Tree x: xs ) {
- String v = pattern_head(x);
- pvars = pvars.append(#<`v>);
- if (depth(v) == level)
- group_by_vars = group_by_vars.cons(#<`v>);
- else rest = rest.append(#<`v>);
- };
- if (!rest.is_empty()) {
- if (pvars.is_empty())
- return final_groupBy(level+1,rest,plan);
- Tree nv = new_var();
- Tree tp = (pvars.length()==1) ? pvars.head() : #<tuple(...pvars)>;
- Tree tg = ordered_tuple(group_by_vars);
- Tree tr = ordered_tuple(rest);
- Tree new_plan = #<groupBy(cmap(lambda(`tp,bag(tuple(`tg,`tr))),`plan))>;
- Tree p = final_groupBy(level+1,rest,nv);
- return #<cmap(lambda(tuple(`tg,`nv),bag(tuple(`tg,`p))),`new_plan)>;
- } else return plan;
- }
-
- static Tree final_groupBy ( Tree plan, Tree pattern ) {
- match pattern {
- case `f(...r):
- return final_groupBy(1,r,plan);
- };
- throw new Error("Wrong pattern in final group-by: "+pattern);
- }
-
- /** plan cost */
- static double cost ( int i, int j ) {
- return size[i]*size[j]*selectivity[i][j];
- }
-
- public static Tree best_plan ( Tree e ) {
- Trees binds = all_binds(e,#[]);
- if (binds.equals(#[]))
- return e;
- int N = binds.length();
- if (N==0)
- return e;
- predicate = new Tree[N][];
- selectivity = new double[N][];
- plan = new Tree[N];
- var = new String[N];
- variables = new BitSet[N];
- pattern = new Tree[N];
- depth = new int[N];
- size = new double[N];
- depends = new BitSet[N];
- filter = new Tree[N];
- depths = new HashMap<String,Integer>();
- Trees al = binds;
- for ( int i = 0; i < N; i++, al = al.tail() ) {
- match al.head() {
- case bind(`v,`d):
- var[i] = ((VariableLeaf) v).value();
- variables[i] = new BitSet();
- variables[i].set(i);
- pattern[i] = #<`v>;
- filter[i] = #<true>;
- };
- predicate[i] = new Tree[N];
- selectivity[i] = new double[N];
- for ( int j = 0; j < N; j++ ) {
- predicate[i][j] = #<true>;
- selectivity[i][j] = 1.0;
- };
- depends[i] = new BitSet();
- };
- header_binds = new SymbolTable();
- query_variables = #[];
- Header h = build_graph(e,0);
- for ( int i = 0; i < N; i++ ) {
- if (depends[i].isEmpty())
- size[i] = 1000;
- else size[i] = 100;
- };
- header = h.header;
- if (Config.trace) {
- System.out.println("Optimizing MRQL query:\n"+e.pretty(0));
- System.out.println("Query Header:\n"+header.pretty(0));
- System.out.println("Query bindings:");
- header_binds.display();
- System.out.print("Variable/nesting: ");
- for (String k: depths.keySet())
- System.out.print(k+"/"+depths.get(k)+" ");
- System.out.println();
- dump(N);
- };
- no_grouping = false;
- for ( int n = N; n > 1; n-- ) {
- int mi = -1;
- int mj = -1;
- double min = Double.MAX_VALUE;
- for ( int i = 0; i < n; i++ )
- for ( int j = 0; j < n; j++ ) {
- if (i != j && eligible(i,j,n)) {
- double cost = cost(i,j);
- if (Config.trace)
- System.out.println("Cost "+i+" "+j+" = "+cost);
- if (cost < min) {
- min = cost;
- mi = i;
- mj = j;
- }
- }
- };
- if (mi < 0 || mj < 0) {
- // irreducible graph;
- // from now on, we operate without grouping and we group-by at the end
- if (Config.trace)
- System.out.println("Switching to flat mode (no grouping during operations)");
- no_grouping = true;
- n++;
- continue;
- };
- if (Config.trace)
- System.out.println("Reduce "+mi+" with "+mj+" into "+mi);
- // merge node mi with node mj into node mi
- plan[mi] = make_plan(mi,mj);
- depth[mi] = Math.min(depth[mi],depth[mj]);
- variables[mi].or(variables[mj]);
- size[mi] = size[mi]*size[mj]*selectivity[mi][mj];
- depends[mi].clear(mj);
- filter[mi] = #<true>;
- for ( int k = 0; k < n; k++ )
- if (k != mi) {
- selectivity[mi][k] = selectivity[k][mi] = selectivity[mi][k]*selectivity[mj][k];
- predicate[mi][k] = predicate[k][mi] = and(predicate[mi][k],predicate[mj][k]);
- if (depends[k].get(mj)) {
- depends[k].clear(mj);
- depends[k].set(mi);
- }
- };
- // replace node mj with node n-1 (last node)
- plan[mj] = plan[n-1];
- depth[mj] = depth[n-1];
- pattern[mj] = pattern[n-1];
- filter[mj] = filter[n-1];
- depends[mj] = depends[n-1];
- variables[mj] = variables[n-1];
- for ( int k = 0; k < n-1; k++ )
- if (k != mj) {
- selectivity[mj][k] = selectivity[k][mj] = selectivity[n-1][k];
- predicate[mj][k] = predicate[k][mj] = predicate[n-1][k];
- if (depends[k].get(n-1)) {
- depends[k].clear(n-1);
- depends[k].set(mj);
- }
- };
- size[mj] = size[n-1];
- if (Config.trace)
- dump(n-1);
- };
- // forced group-by
- if (no_grouping) {
- plan[0] = final_groupBy(plan[0],pattern[0]);
- if (h.pattern.length() == 1)
- match h.pattern.head() {
- case bind(`v,`p):
- return Meta.subst_expr(v,plan[0],h.header);
- };
- };
- Tree np = pattern(pattern[0],false);
- if (h.pattern.length() == 1)
- match h.pattern.head() {
- case bind(`v,`p):
- return subst_header(v,np,plan[0],h.header);
- };
- return e;
- }
-}
-
- private static Tree process_repeat_plan ( Tree e ) {
- match e {
- case repeat(lambda(`x,`step),`init,...r):
- Tree ns = SingleQueryPlan.best_plan(step);
- repeat_plans.put(x.toString(),ns);
- return #<repeat(lambda(`x,step(`x)),`init,...r)>;
- case closure(lambda(`x,`step),`init,...r):
- Tree ns = SingleQueryPlan.best_plan(step);
- repeat_plans.put(x.toString(),ns);
- return #<closure(lambda(`x,cstep(`x)),`init,...r)>;
- case `f(...al):
- Trees bl = #[];
- for (Tree a: al)
- bl = bl.append(process_repeat_plan(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- private static Tree process_nested_plan ( Tree e ) {
- match e {
- case select(`u,from(...bl),where(`p)):
- return SingleQueryPlan.best_plan(e);
- case `f(...al):
- Trees bl = #[];
- for (Tree a: al)
- bl = bl.append(process_nested_plan(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- public static Tree best_plan ( Tree e ) {
- repeat_plans = new Hashtable<String,Tree>();
- Tree np = process_nested_plan(process_repeat_plan(e));
- for ( String s: repeat_plans.keySet() )
- np = Meta.subst_expr(#<step(`s)>,repeat_plans.get(s),
- Meta.subst_expr(#<cstep(`s)>,repeat_plans.get(s),np));
- return np;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Simplification.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Simplification.gen b/src/main/java/core/Simplification.gen
deleted file mode 100644
index f18b17a..0000000
--- a/src/main/java/core/Simplification.gen
+++ /dev/null
@@ -1,389 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.*;
-import java.io.*;
-
-
-/** simplify algebraic forms using heuristic rewriting rules that improve performance in most cases */
-public class Simplification extends Normalization {
-
- /* true if x is functional dependent on y (ie, equal x's implies equal y's) */
- private static boolean functional_dependent ( Tree x, Tree y ) {
- if (x.equals(y))
- return true;
- match y {
- case tuple(...ts):
- for ( Tree t: ts )
- if (functional_dependent(x,t))
- return true;
- case record(...rs):
- for ( Tree r: rs )
- match r {
- case bind(_,`t):
- if (functional_dependent(x,t))
- return true;
- }
- };
- return false;
- }
-
- private static boolean simple_accessor ( Tree v, Tree e ) {
- match e {
- case nth(`u,`n): return simple_accessor(v,u);
- case project(`u,`a): return simple_accessor(v,u);
- };
- return e.equals(v);
- }
-
- private static Trees factor_out_aggr ( Tree e, Tree v ) {
- match e {
- case call(`g,cmap(`f,`d)):
- if (!simple_accessor(v,d) || !free_variables(f,#[]).is_empty())
- fail;
- for ( Tree monoid: monoids )
- match monoid {
- case `aggr(...):
- if (aggr.equals(g.toString()))
- return #[bind(`e,`d)];
- };
- fail
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(factor_out_aggr(a,v));
- return bl;
- };
- return #[];
- }
-
- private static Trees factor_out_aggregations ( Tree e, Tree v ) {
- Trees al = factor_out_aggr(e,v);
- Tree t = e;
- for ( Tree a: al )
- match a {
- case bind(`u,`d): t = subst(u,#<0>,t);
- };
- Trees bl = #[];
- for ( Tree a: al )
- match a {
- case bind(`u,`d):
- if (occurences(d,t) == 0) {
- boolean exists = false;
- for ( Tree b: bl )
- match b {
- case bind(`uu,`dd):
- if (d.equals(dd) && !u.equals(uu))
- if (alpha_equivalent(u,uu))
- exists = true;
- };
- if (true || !exists)
- bl = bl.append(a);
- }
- };
- return bl;
- }
-
- /** Algebraic normalization (algebra to algebra)
- * @param e algebraic expression
- * @return an improved algebraic expression
- */
- public static Tree simplify ( Tree e ) {
- match e {
- case cmap(`f,cmap(lambda(`x,`g),`s)):
- return simplify(#<cmap(lambda(`x,cmap(`f,`g)),`s)>);
- case map(`f,cmap(lambda(`x,`g),`s)):
- return simplify(#<cmap(lambda(`x,map(`f,`g)),`s)>);
- case cmap(`g,join(`k1,`k2,lambda(`p,`f),`X,`Y)):
- return simplify(#<join(`k1,`k2,lambda(`p,cmap(`g,`f)),`X,`Y)>);
- case cmap(lambda(`x,`S(`y)),`u):
- if (is_collection(S) && x.equals(y))
- return simplify(u);
- else fail
- case cmap(lambda(`x,`b),`S(`a)):
- if (is_collection(S) && x.is_variable())
- return simplify(subst_var(x,a,b));
- else fail
- case cmap(`f,`S()):
- if (is_collection(S))
- return #<`S()>;
- else fail
- case cmap(lambda(`x,`T(`b)),`S(...as)):
- if (is_collection(S) && is_collection(T) && x.is_variable()) {
- Trees bs = #[];
- for ( Tree a: as )
- bs = bs.append(simplify(subst_var(x,a,b)));
- return #<`T(...bs)>;
- } else fail
- case map(lambda(`x,`b),`S(`a)):
- if (is_collection(S) && x.is_variable())
- return #<`S(`(simplify(subst_var(x,a,b))))>;
- else fail
- case map(`f,`S()):
- if (is_collection(S))
- return #<`S()>;
- else fail
- case filter(lambda(`x,`b),`m,`S(`a)):
- if (is_collection(S) && x.is_variable())
- return simplify(#<if(`(subst_var(x,a,b)),apply(`m,`a),`S())>);
- else fail
- case filter(`p,`m,`S()):
- if (is_collection(S))
- return #<`S()>;
- else fail
- case cmap(`f,if(`p,`x,`y)):
- return simplify(#<if(`p,cmap(`f,`x),cmap(`f,`y))>);
- // if the join reducer contains an independent aggregation push it to the input
- // NOTE: This disables the self-join to MR transformation used in pagerank
- case xxxjoin(`kx,`ky,
- lambda(`v,cmap(lambda(`v1,cmap(lambda(`v2,bag(tuple(`k,`b))),
- nth(`vy,1))),
- nth(`vx,0))),
- `X,`Y):
- if (!vx.equals(v) || !vy.equals(v))
- fail;
- Trees l1 = factor_out_aggregations(b,v1);
- Trees l2 = factor_out_aggregations(b,v2);
- if (l1.is_empty() && l2.is_empty())
- fail;
- Tree px = v1;
- Trees dl = #[];
- for ( Tree a: l1 )
- match a {
- case bind(`u,`d):
- Tree nv = new_var();
- b = subst(u,nv,b);
- if (true || !dl.member(d)) {
- Tree vv = new_var();
- X = #<cmap(lambda(`px,bag(tuple(`px,`u))),`X)>;
- px = #<tuple(`px,`nv)>;
- }
- dl = dl.append(d);
- };
- Tree py = v2;
- dl = #[];
- for ( Tree a: l2 )
- match a {
- case bind(`u,`d):
- Tree nv = new_var();
- b = subst(u,nv,b);
- if (true || !dl.member(d)) {
- Tree vv = new_var();
- Y = #<cmap(lambda(`py,bag(tuple(`py,`u))),`Y)>;
- py = #<tuple(`py,`nv)>;
- };
- dl = dl.append(d);
- };
- Tree res = #<join(lambda(`px,apply(`kx,`v1)),
- lambda(`py,apply(`ky,`v2)),
- lambda(`v,cmap(lambda(`px,cmap(lambda(`py,bag(tuple(`k,`b))),
- nth(`vy,1))),
- nth(`vx,0))),
- `X,`Y)>;
- res = rename(res);
- return simplify(res);
- // if the reducer of a join generates pairs (k,v), where k is functional dependent
- // on a join key, then the outer groupBy just groups the v values
- case groupBy(join(lambda(`vx,`bx),`ky,
- lambda(`v,cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ex,`br))),
- nth(`v1,1))),
- nth(`v2,0))),
- `X,`Y)):
- if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vx,x,bx),ex))
- return simplify(#<join(lambda(`vx,`bx),`ky,
- lambda(`v,groupBy(cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ex,`br))),
- nth(`v1,1))),
- nth(`v2,0)))),
- `X,`Y)>);
- fail
- // same for the right key
- case groupBy(join(`kx,lambda(`vy,`by),
- lambda(`v,cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ey,`br))),
- nth(`v1,1))),
- nth(`v2,0))),
- `X,`Y)):
- if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vy,y,by),ey))
- return simplify(#<join(`kx,lambda(`vy,`by),
- lambda(`v,groupBy(cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ey,`br))),
- nth(`v1,1))),
- nth(`v2,0)))),
- `X,`Y)>);
- fail
- // same for the left key, different nesting
- case groupBy(join(lambda(`vx,`bx),`ky,
- lambda(`v,cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ex,`br))),
- nth(`v1,0))),
- nth(`v2,1))),
- `X,`Y)):
- if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vx,x,bx),ex))
- return simplify(#<join(lambda(`vx,`bx),`ky,
- lambda(`v,groupBy(cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ex,`br))),
- nth(`v1,1))),
- nth(`v2,0)))),
- `X,`Y)>);
- fail
- // same for the right key, different nesting
- case groupBy(join(`kx,lambda(`vy,`by),
- lambda(`v,cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ey,`br))),
- nth(`v1,0))),
- nth(`v2,1))),
- `X,`Y)):
- if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vy,y,by),ey))
- return simplify(#<join(`kx,lambda(`vy,`by),
- lambda(`v,groupBy(cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ey,`br))),
- nth(`v1,0))),
- nth(`v2,1)))),
- `X,`Y)>);
- fail
- // same for the left key, right nested
- case groupBy(join(lambda(`vx,`bx),`ky,
- lambda(`v,cmap(lambda(`x,bag(tuple(`ex,`br))),
- nth(`v1,0))),
- `X,`Y)):
- if (v1.equals(v) && functional_dependent(subst(vx,x,bx),ex))
- return simplify(#<join(lambda(`vx,`bx),`ky,
- lambda(`v,groupBy(cmap(lambda(`x,bag(tuple(`ex,`br))),
- nth(`v1,0)))),
- `X,`Y)>);
- fail
- // same for the right key, left nested
- case groupBy(join(`kx,lambda(`vy,`by),
- lambda(`v,cmap(lambda(`y,bag(tuple(`ey,`br))),
- nth(`v2,1))),
- `X,`Y)):
- if (v2.equals(v) && functional_dependent(subst(vy,y,by),ey))
- return simplify(#<join(`kx,lambda(`vy,`by),
- lambda(`v,groupBy(cmap(lambda(`y,bag(tuple(`ey,`br))),
- nth(`v2,1)))),
- `X,`Y)>);
- fail
- // if we group-by the join key, then embed the group-by in the join reducer
- // (redundant rule)
- case groupBy(join(`kx,`ky,lambda(`v,cmap(lambda(`v1,cmap(lambda(`v2,bag(tuple(`k,`u))),`e1)),`e2)),
- `X,`Y)):
- if (((e1.equals(#<nth(`v,0)>) && e2.equals(#<nth(`v,1)>))
- || (e2.equals(#<nth(`v,0)>) && e1.equals(#<nth(`v,1)>)))
- && (alpha_equivalent(kx,#<lambda(`v1,`k)>)
- || alpha_equivalent(kx,#<lambda(`v2,`k)>)))
- return simplify(#<join(`kx,`ky,lambda(`v,groupBy(cmap(lambda(`v1,cmap(lambda(`v2,
- bag(tuple(`k,`u))),`e1)),`e2))),
- `X,`Y)>);
- fail
- case groupBy(groupBy(`x)):
- Tree nv = new_var();
- return simplify(#<cmap(lambda(`nv,bag(bag(`nv))),groupBy(`x))>);
- case repeat(lambda(`v,`b),`s,...l):
- repeat_variables = repeat_variables.cons(v);
- return #<repeat(lambda(`v,`(simplify(b))),`(simplify(s)),...l)>;
- case closure(lambda(`v,`b),`s,...l):
- repeat_variables = repeat_variables.cons(v);
- return #<closure(lambda(`v,`(simplify(b))),`(simplify(s)),...l)>;
- case loop(lambda(tuple(...vs),`b),`s,`n):
- repeat_variables = repeat_variables.append(vs);
- return #<loop(lambda(tuple(...vs),`(simplify(b))),`(simplify(s)),`n)>;
- case aggregate(`acc,`zero,`T()):
- if (is_collection(T))
- return zero;
- else fail
- case aggregate(`acc,`zero,`T(`s)):
- if (is_collection(T))
- return simplify(#<apply(`acc,tuple(`zero,`s))>);
- else fail
- case apply(lambda(`v,`b),`u):
- if (!v.is_variable())
- fail;
- return simplify(subst_var(v,u,b));
- case apply(function(tuple(...el),_,`b),`u):
- int i = 0;
- for ( Tree a: el )
- match a {
- case `bind(`v,_):
- b = subst(v,#<nth(`u,`(i++))>,b);
- };
- return simplify(b);
- case call(and,true,`u): return simplify(u);
- case call(and,`u,true): return simplify(u);
- case call(and,false,`u):return #<false>;
- case call(and,`u,false): return #<false>;
- case call(or,true,`u): return #<true>;
- case call(or,`u,true): return #<true>;
- case call(or,false,`u): return simplify(u);
- case call(or,`u,false): return simplify(u);
- case call(not,true): return #<false>;
- case call(not,false): return #<true>;
- case if(true,`e1,`e2): return simplify(e1);
- case if(false,`e1,`e2): return simplify(e2);
- case call(count,cmap(lambda(`v,`S(`x)),`u)):
- if (is_collection(S))
- return simplify(#<call(count,`u)>);
- else fail
- case call(count,`groupBy(cmap(lambda(`v,`S(tuple(`x,`y))),`u))):
- if (is_collection(S) && !y.equals(#<0>) && #[groupBy,orderBy].member(#<`groupBy>))
- return #<call(count,groupBy(cmap(lambda(`v,`S(tuple(`x,0))),`u)))>;
- else fail
- case call(count,`S(...r)):
- if (is_collection(S))
- return #<typed(`(r.length()),long)>;
- else fail
- case call(`f,`S(`x)):
- if (!is_collection(S))
- fail;
- for ( Tree m: monoids )
- match m {
- case `aggr(`mtp,`plus,`zero,`unit):
- if (!aggr.equals(f.toString()))
- continue;
- if (TypeInference.unify(mtp,TypeInference.type_inference2(x)) != null)
- return simplify(#<apply(`unit,`x)>);
- };
- fail
- case nth(tuple(...al),`n):
- if (!n.is_long())
- fail;
- int i = (int)n.longValue();
- if (i >= 0 && i < al.length())
- return simplify(al.nth(i));
- case project(record(...bl),`a):
- for ( Tree b: bl )
- match b {
- case bind(`v,`u):
- if (v.equals(a))
- return simplify(u);
- };
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(simplify(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- /** Algebraic normalization (algebra to algebra) applied multiple times
- * @param e algebraic expression
- * @return an improved algebraic expression
- */
- public static Tree simplify_all ( Tree e ) {
- Tree ne = simplify(e);
- if (e.equals(ne))
- return e;
- else return simplify_all(ne);
- }
-}
[23/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Environment.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Environment.java b/core/src/main/java/org/apache/mrql/Environment.java
new file mode 100644
index 0000000..da8dd84
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Environment.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.mrql.gen.Tree;
+
+
+/** the run-time environment for in-memory evaluation (binds variables to MRData) */
+final public class Environment implements Serializable {
+ public String name;
+ public MRData value;
+ public Environment next;
+
+ Environment ( String n, MRData v, Environment next ) {
+ name = n;
+ value = v;
+ this.next = next;
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ out.defaultWriteObject();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ name = Tree.add(name);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Evaluator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Evaluator.java b/core/src/main/java/org/apache/mrql/Evaluator.java
new file mode 100644
index 0000000..28d356f
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Evaluator.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java_cup.runtime.*;
+import org.apache.mrql.gen.*;
+import java.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+
+
+/** Evaluates physical plans using one of the evaluation engines */
+abstract public class Evaluator extends Interpreter {
+
+ /** the current MRQL evaluator */
+ public static Evaluator evaluator;
+
+ /** initialize the evaluator */
+ abstract public void init ( Configuration conf );
+
+ /** shutdown the evaluator */
+ abstract public void shutdown ( Configuration conf );
+
+ /** initialize the query evaluation */
+ abstract public void initialize_query ();
+
+ /** create a new evaluation configuration */
+ abstract public Configuration new_configuration ();
+
+ /** synchronize peers in BSP mode */
+ public MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
+ throw new Error("You can only synchronize BSP tasks");
+ }
+
+ /** distribute a bag among peers in BSP mode */
+ public Bag distribute ( MR_string peerName, Bag s ) {
+ throw new Error("You can only distribute bags among BSP tasks");
+ }
+
+ /** run a BSP task */
+ public MRData bsp ( Tree plan, Environment env ) throws Exception {
+ throw new Error("You can only run a BSP task in BSP mode");
+ }
+
+ /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
+ abstract public Class<? extends MRQLFileInputFormat> parsedInputFormat ();
+
+ /** return the FileInputFormat for binary files */
+ abstract public Class<? extends MRQLFileInputFormat> binaryInputFormat ();
+
+ /** return the FileInputFormat for data generator files */
+ abstract public Class<? extends MRQLFileInputFormat> generatorInputFormat ();
+
+ /** The Aggregate physical operator
+ * @param acc_fnc the accumulator function from (T,T) to T
+ * @param zero the zero element of type T
+ * @param plan the plan that constructs the dataset that contains the bag of values {T}
+ * @param env contains bindings fro variables to values (MRData)
+ * @return the aggregation result of type T
+ */
+ abstract public MRData aggregate ( Tree acc_fnc,
+ Tree zero,
+ Tree plan,
+ Environment env ) throws Exception;
+
+ /** Evaluate a loop a fixed number of times */
+ abstract public Tuple loop ( Tree e, Environment env ) throws Exception;
+
+ /** Evaluate a MRQL physical plan and print tracing info
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a DataSet (stored in HDFS)
+ */
+ abstract public DataSet eval ( final Tree e,
+ final Environment env,
+ final String counter );
+
+ final static MR_long counter_key = new MR_long(0);
+ final static MRContainer counter_container = new MRContainer(counter_key);
+ final static MRContainer value_container = new MRContainer(new MR_int(0));
+
+ /** dump MRQL data into a sequence file */
+ public void dump ( String file, Tree type, MRData data ) throws Exception {
+ Path path = new Path(file);
+ FileSystem fs = path.getFileSystem(Plan.conf);
+ PrintStream ftp = new PrintStream(fs.create(path.suffix(".type")));
+ ftp.print("2@"+type.toString()+"\n");
+ ftp.close();
+ SequenceFile.Writer writer
+ = new SequenceFile.Writer(fs,Plan.conf,path,
+ MRContainer.class,MRContainer.class);
+ if (data instanceof MR_dataset)
+ data = Plan.collect(((MR_dataset)data).dataset());
+ if (data instanceof Bag) {
+ Bag s = (Bag)data;
+ long i = 0;
+ for ( MRData e: s ) {
+ counter_key.set(i++);
+ value_container.set(e);
+ writer.append(counter_container,value_container);
+ }
+ } else {
+ counter_key.set(0);
+ value_container.set(data);
+ writer.append(counter_container,value_container);
+ };
+ writer.close();
+ }
+
+ /** dump MRQL data into a text CVS file */
+ public void dump_text ( String file, Tree type, MRData data ) throws Exception {
+ int ps = Config.max_bag_size_print;
+ Config.max_bag_size_print = -1;
+ final PrintStream out = (Config.hadoop_mode)
+ ? Plan.print_stream(file)
+ : new PrintStream(file);
+ if (data instanceof MR_dataset)
+ data = Plan.collect(((MR_dataset)data).dataset());
+ if (Translator.collection_type(type)) {
+ Tree tp = ((Node)type).children().head();
+ if (tp instanceof Node && ((Node)tp).name().equals("tuple")) {
+ Trees ts = ((Node)tp).children();
+ for ( MRData x: (Bag)data ) {
+ Tuple t = (Tuple)x;
+ out.print(print(t.get((short)0),ts.nth(0)));
+ for ( short i = 1; i < t.size(); i++ )
+ out.print(","+print(t.get(i),ts.nth(i)));
+ out.println();
+ }
+ } else for ( MRData x: (Bag)data )
+ out.println(print(x,tp));
+ } else out.println(print(data,query_type));
+ Config.max_bag_size_print = ps;
+ out.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Function.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Function.java b/core/src/main/java/org/apache/mrql/Function.java
new file mode 100644
index 0000000..1f1d4e1
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Function.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+
+
+/**
+ * An anonymous function from MRData to MRData (a lambda abstraction)
+ * Must provide a concrete implementation for eval (the lambda body)
+ */
+abstract public class Function implements Serializable {
+ /**
+ * Evaluate the anonymous function from MRData to MRData
+ * @param arg the operand to be evaluated
+ * @return the result of evaluating
+ */
+ abstract public MRData eval ( final MRData arg );
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/GeneratorDataSource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/GeneratorDataSource.java b/core/src/main/java/org/apache/mrql/GeneratorDataSource.java
new file mode 100644
index 0000000..380f50d
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/GeneratorDataSource.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** A DataSource used for processing the range min..max */
+final public class GeneratorDataSource extends DataSource {
+ GeneratorDataSource ( int source_num, String path, Configuration conf ) {
+ super(source_num,path,Evaluator.evaluator.generatorInputFormat(),conf);
+ }
+
+ GeneratorDataSource ( String path, Configuration conf ) {
+ super(-1,path,Evaluator.evaluator.generatorInputFormat(),conf);
+ }
+
+ public static long size ( Path path, Configuration conf ) throws IOException {
+ // each file generates range_split_size long integers
+ FileStatus s = path.getFileSystem(conf).getFileStatus(path);
+ if (!s.isDir())
+ return Config.range_split_size*8;
+ long size = 0;
+ for ( FileStatus fs: path.getFileSystem(conf).listStatus(path) )
+ size += Config.range_split_size*8;
+ return size;
+ }
+
+ public String toString () {
+ return "Generator"+separator+source_num+separator+path;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Interpreter.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Interpreter.gen b/core/src/main/java/org/apache/mrql/Interpreter.gen
new file mode 100644
index 0000000..97c76ce
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Interpreter.gen
@@ -0,0 +1,833 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.io.PrintStream;
+
+
+/** The MRQL interpreter */
+public class Interpreter extends TypeInference {
+
+ public final static Tree identity_mapper = #<lambda(x,bag(x))>;
+
+ protected static Environment global_env = null;
+
+ /** retrieve variable binding */
+ public final static MRData variable_lookup ( final String v, final Environment environment ) {
+ for ( Environment env = environment; env != null; env = env.next ) {
+ if (v.equals(env.name))
+ return env.value;
+ };
+ return null;
+ }
+
+ /** insert a new global variable binding */
+ public final static void new_global_binding ( final String var, final MRData value ) {
+ if (value instanceof Bag)
+ ((Bag)value).materialize();
+ global_env = new Environment(var,value,global_env);
+ }
+
+ /** remove a global variable binding */
+ public static void remove_global_binding ( String v ) {
+ if (global_env == null)
+ return;
+ for ( Environment env = global_env; env.next != null; env = env.next )
+ if (v.equals(env.next.name))
+ env.next = env.next.next;
+ if (global_env.name == v)
+ global_env = global_env.next;
+ }
+
+ /** retrieve a global variable binding */
+ public static MRData lookup_global_binding ( String v ) {
+ for ( Environment env = global_env; env != null; env = env.next )
+ if (v.equals(env.name))
+ return env.value;
+ return null;
+ }
+
+ public static void set_global_bindings ( Environment env ) {
+ global_env = env;
+ }
+
+ final static int coerce_method = ClassImporter.find_method_number("coerce",#[any,int]);
+
+ /** untyped reify: not type-correct but will not crash the run-time system */
+ private final static Tree reify ( final MRData x ) {
+ if (x instanceof Bag) {
+ Bag b = (Bag)x;
+ Trees as = #[];
+ for ( MRData e: b)
+ as = as.append(reify(e));
+ return #<list(...as)>;
+ } else if (x instanceof Tuple) {
+ Tuple t = (Tuple)x;
+ Trees as = #[];
+ for ( short i = 0; i < t.size(); i++ )
+ as = as.append(reify(t.get(i)));
+ return #<tuple(...as)>;
+ } else if (x instanceof MR_string)
+ return new StringLeaf(((MR_string)x).get());
+ else if (x instanceof MR_short)
+ return #<callM(coerce,`coerce_method,`(((MR_short)x).get()),`(MRContainer.SHORT))>;
+ else if (x instanceof MR_int)
+ return #<`(((MR_int)x).get())>;
+ else if (x instanceof MR_long)
+ return #<callM(coerce,`coerce_method,`((int)((MR_long)x).get()),`(MRContainer.LONG))>;
+ else if (x instanceof MR_float)
+ return #<`(((MR_float)x).get())>;
+ else if (x instanceof MR_double)
+ return #<callM(coerce,`coerce_method,`((float)(((MR_double)x).get())),`(MRContainer.DOUBLE))>;
+ throw new Error("wrong MRData: "+x);
+ }
+
+ /** evaluate an MRQL function in memory */
+ private final static Function evalf ( final String v,
+ final Tree body,
+ final Environment env ) {
+ return new Function() {
+ final public MRData eval ( final MRData x ) {
+ return evalE(body,new Environment(v,x,env));
+ }
+ };
+ }
+
+ /** evaluate an MRQL function in memory */
+ public final static Function evalF ( Tree fnc, Environment env ) {
+ match fnc {
+ case compiled(`ln,`lm,...vars):
+ try {
+ return Compiler.compiled(Thread.currentThread().getContextClassLoader(),ln.toString());
+ } catch (Exception ex) {
+ System.err.println("*** Unable to retrieve the compiled lambda: "+fnc);
+ return ((Lambda) evalE(lm)).lambda();
+ }
+ case lambda(`v,`b):
+ return evalf(v.toString(),b,env);
+ case function(tuple(...params),`tp,`body):
+ String[] as = new String[params.length()];
+ int i = 0;
+ for ( Tree param: params )
+ match param {
+ case `bind(`v,_):
+ as[i++] = v.toString();
+ };
+ return evalT(as,body,env);
+ };
+ throw new Error("Ill-formed lambda: "+fnc);
+ }
+
+ /** evaluate an MRQL function in memory */
+ private final static Function evalT ( final String[] params,
+ final Tree body,
+ final Environment env ) {
+ return new Function() {
+ final public MRData eval ( final MRData x ) {
+ Environment new_env = env;
+ for ( int i = 0; i < params.length; i++ )
+ new_env = new Environment(params[i],((Tuple)x).get(i),new_env);
+ return evalE(body,new_env);
+ }
+ };
+ }
+
+ final static String true_name = #<true>.toString();
+ final static String false_name = #<false>.toString();
+ final static String null_name = #<null>.toString();
+ final static MRData null_value = new Tuple(0);
+ final static MRData true_value = new MR_bool(true);
+ final static MRData false_value = new MR_bool(false);
+
+ static int tab_count = -3;
+
+ public static String tabs ( int n ) {
+ StringBuffer b = new StringBuffer();
+ for ( int i = 0; i < n; i++)
+ b.append(' ');
+ return b.toString();
+ }
+
+ /** evaluate an MRQL expression in memory and print tracing info */
+ final static MRData evalE ( final Tree e, final Environment env ) {
+ if (Config.trace_exp_execution) {
+ tab_count += 3;
+ System.out.println(tabs(tab_count)+print_query(e));
+ };
+ MRData res = evalEE(e,env);
+ if (Config.trace_exp_execution) {
+ System.out.println(tabs(tab_count)+"-> "+res);
+ tab_count -= 3;
+ };
+ return res;
+ }
+
+ /** evaluate an MRQL expression in memory */
+ private final static MRData evalEE ( final Tree e, final Environment env ) {
+ try {
+ if (e.is_variable()) {
+ String v = e.toString();
+ if (v == true_name)
+ return true_value;
+ else if (v == false_name)
+ return false_value;
+ else if (v == null_name)
+ return null_value;
+ MRData x = variable_lookup(v,env);
+ if (x != null)
+ return x;
+ x = lookup_global_binding(v);
+ if (x == null)
+ throw new Error("Variable "+v+" is not bound");
+ return x;
+ } else if (e.is_long())
+ return new MR_int((int)e.longValue());
+ else if (e.is_double())
+ return new MR_float((float)e.doubleValue());
+ else if (e.is_string())
+ return new MR_string(e.stringValue());
+ match e {
+ case callM(and,_,`x,`y): // lazy
+ return (((MR_bool)evalE(x,env)).get()) ? evalE(y,env) : false_value;
+ case callM(or,_,`x,`y):
+ return (((MR_bool)evalE(x,env)).get()) ? true_value : evalE(y,env);
+ case callM(`f,`n,...args): // internal function call
+ MRData[] as = new MRData[args.length()];
+ for ( int i = 0; i < args.length(); i++ )
+ as[i] = evalE(args.nth(i),env);
+ return ClassImporter.call((int)n.longValue(),as);
+ case compiled(`ln,_,...vars):
+ return new Lambda(Compiler.compiled(Thread.currentThread().getContextClassLoader(),ln.toString()));
+ case lambda(`v,`body):
+ return new Lambda(evalf(v.toString(),body,env));
+ case nth(`x,`n):
+ return ((Tuple)evalE(x,env)).get((int)n.longValue());
+ case setNth(`x,`n,`v,`ret):
+ return ((Tuple)evalE(x,env)).set((int)n.longValue(),evalE(v,env),evalE(ret,env));
+ case materialize(`u):
+ return MapReduceAlgebra.materialize(evalE(u,env));
+ case let(`v,`u,`body):
+ MRData x = evalE(u,env);
+ if (x instanceof Bag)
+ ((Bag)x).materialize();
+ return evalE(body,new Environment(v.toString(),x,env));
+ case cmap(`f,`s):
+ return MapReduceAlgebra.cmap(evalF(f,env),(Bag)evalE(s,env));
+ case filter(`p,`m,`s):
+ return MapReduceAlgebra.filter(evalF(p,env),evalF(m,env),(Bag)evalE(s,env));
+ case map(`m,`s):
+ return MapReduceAlgebra.map(evalF(m,env),(Bag)evalE(s,env));
+ case repeat(lambda(`v,`b),`s,`n):
+ final String nm = v.toString();
+ final Tree body = b;
+ if (Config.hadoop_mode) {
+ Function loop_fnc = new Function () {
+ public MRData eval ( MRData s ) {
+ new_global_binding(nm,s);
+ MRData val = new MR_dataset(Evaluator.evaluator.eval(body,new Environment(nm,s,env),nm));
+ return val;
+ }; };
+ return MapReduceAlgebra.repeat(loop_fnc,(Bag)evalE(s,env),((MR_int)evalE(n,env)).get());
+ } else {
+ Function loop_fnc = new Function () {
+ public MRData eval ( MRData s ) {
+ new_global_binding(nm,s);
+ return evalM(body,new Environment(nm,s,env));
+ }; };
+ return MapReduceAlgebra.repeat(loop_fnc,(Bag)evalE(s,env),((MR_int)evalE(n,env)).get());
+ }
+ case repeat(`lm,`s,`n):
+ return MapReduceAlgebra.repeat(evalF(lm,env),(Bag)evalE(s,env),((MR_int)evalE(n,env)).get());
+ case range(`min,`max):
+ return MapReduceAlgebra.generator(((MR_long)evalE(min,env)).get(),
+ ((MR_long)evalE(max,env)).get());
+ case call(`f,...args):
+ Tuple t = new Tuple(args.length());
+ int i = 0;
+ for ( Tree a: args )
+ t.set(i++,evalE(a,env));
+ return evalF(f,env).eval(t);
+ case tuple(`x,`y):
+ return new Tuple(evalE(x,env),evalE(y,env));
+ case tuple(`x,`y,`z):
+ return new Tuple(evalE(x,env),evalE(y,env),evalE(z,env));
+ case tuple(...el):
+ Tuple t = new Tuple(el.length());
+ int i = 0;
+ for ( Tree a: el )
+ t.set(i++,evalE(a,env));
+ return t;
+ case tagged_union(`n,`u):
+ return new Union((byte)n.longValue(),evalE(u,env));
+ case union_value(`x):
+ return ((Union)evalE(x,env)).value();
+ case union_tag(`x):
+ return new MR_int(((Union)evalE(x,env)).tag());
+ // used for shortcutting sync in bsp supersteps
+ case BAG():
+ return SystemFunctions.bsp_empty_bag;
+ case TRUE():
+ return SystemFunctions.bsp_true_value;
+ case FALSE():
+ return SystemFunctions.bsp_false_value;
+ case `T(...el):
+ if (!is_collection(T))
+ fail;
+ if (el.is_empty())
+ return new Bag();
+ Bag b = new Bag(el.length());
+ for ( Tree a: el )
+ b.add(evalE(a,env));
+ return b;
+ case if(`c,`x,`y):
+ if (((MR_bool)evalE(c,env)).get())
+ return evalE(x,env);
+ else return evalE(y,env);
+ case Collect(`s):
+ try {
+ if (Config.hadoop_mode)
+ return Plan.collect(Evaluator.evaluator.eval(s,env,"-"));
+ Bag b = evalS(s,env);
+ b.materialize();
+ return b;
+ } catch (Exception ex) { throw new Error(ex); }
+ case dataset_size(`x):
+ return new MR_long(Plan.size(Evaluator.evaluator.eval(x,env,"-")) / (1024*1024));
+ case synchronize(`peer,`b):
+ return Evaluator.evaluator.synchronize(((MR_string)evalE(peer,env)),(MR_bool)evalE(b,env));
+ case distribute(`peer,`s):
+ return Evaluator.evaluator.distribute(((MR_string)evalE(peer,env)),(Bag)evalE(s,env));
+ case mapReduce(`m,`r,`s,_):
+ return MapReduceAlgebra.mapReduce(evalF(m,env),
+ evalF(r,env),
+ (Bag)evalE(s,env));
+ case mapReduce2(`mx,`my,`r,`x,`y,_):
+ return MapReduceAlgebra.mapReduce2(
+ evalF(mx,env),
+ evalF(my,env),
+ evalF(r,env),
+ (Bag)evalE(x,env),
+ (Bag)evalE(y,env));
+ case mapJoin(`kx,`ky,`r,`x,`y):
+ return MapReduceAlgebra.mapJoin(
+ evalF(kx,env),
+ evalF(ky,env),
+ evalF(r,env),
+ (Bag)evalE(x,env),
+ (Bag)evalE(y,env));
+ case crossProduct(`mx,`my,`r,`x,`y):
+ return MapReduceAlgebra.crossProduct(
+ evalF(mx,env),
+ evalF(my,env),
+ evalF(r,env),
+ (Bag)evalE(x,env),
+ (Bag)evalE(y,env));
+ case groupBy(`s):
+ return MapReduceAlgebra.groupBy((Bag)evalE(s,env));
+ case orderBy(`s):
+ return MapReduceAlgebra.groupBy((Bag)evalE(s,env));
+ case index(`x,`n):
+ MRData xv = evalE(x,env);
+ MRData nv = evalE(n,env);
+ final int k = (int)((MR_int)nv).get();
+ if (k < 0)
+ throw new Error("Negative list index: "+k);
+ if (xv instanceof MR_dataset) {
+ List<MRData> res = ((MR_dataset)xv).dataset().take(k+1);
+ if (k >= res.size())
+ throw new Error("List index out of bounds: "+k);
+ return res.get(k);
+ };
+ Bag b = (Bag)xv;
+ return b.get(k);
+ case range(`x,`i,`j):
+ MRData xv = evalE(x,env);
+ MRData ni = evalE(i,env);
+ MRData nj = evalE(j,env);
+ int ki = (int)((MR_int)ni).get();
+ int kj = (int)((MR_int)nj).get();
+ if (ki < 0 || kj < ki)
+ throw new Error("Wrong list range: ["+ki+","+kj+"]");
+ Iterator<MRData> it = (xv instanceof MR_dataset)
+ ? ((MR_dataset)xv).dataset().take(kj+1).iterator()
+ : ((Bag)xv).iterator();
+ Bag s = new Bag();
+ for ( int n = 0; it.hasNext() && n < ki; n++ )
+ it.next();
+ for ( int n = ki; it.hasNext() && n <= kj; n++ )
+ s.add(it.next());
+ return s;
+ case map_index(`x,`key):
+ MRData xv = evalE(x,env);
+ final MRData nk = evalE(key,env);
+ if (xv instanceof MR_dataset) {
+ xv = ((MR_dataset)xv).dataset().reduce(new Tuple(),new Function() {
+ public MRData eval ( MRData value ) {
+ Tuple p = (Tuple)value;
+ Tuple y = (Tuple)p.second();
+ return (y.first().equals(nk)) ? y.second() : p.first();
+ }
+ });
+ if (xv instanceof Tuple && ((Tuple)xv).size() == 0)
+ throw new Error("Map key not found: "+nk);
+ return xv;
+ };
+ return ((Bag)xv).map_find(nk);
+ case aggregate(`acc,`zero,`s):
+ return MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
+ (Bag)evalE(s,env));
+ case Aggregate(`acc,`zero,`s):
+ if (Config.hadoop_mode)
+ return Evaluator.evaluator.aggregate(closure(acc,env),zero,s,env);
+ else return MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),evalM(s,env));
+ case mergeGroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
+ return MapReduceAlgebra.mergeGroupByJoin(evalF(kx,env),evalF(ky,env),evalF(gx,env),evalF(gy,env),
+ evalF(m,env),evalF(c,env),evalF(r,env),
+ (Bag)evalE(x,env),(Bag)evalE(y,env));
+ case BSP(tuple(...ns),`superstep,`state,`o,...as):
+ if (Config.hadoop_mode)
+ return Evaluator.evaluator.bsp(e,env);
+ Bag[] ds = new Bag[as.length()];
+ for ( int i = 0; i < ds.length; i++ )
+ ds[i] = evalM(as.nth(i),env);
+ int[] nn = new int[ns.length()];
+ for ( int i = 0; i < ns.length(); i++ )
+ nn[i] = (int)((LongLeaf)ns.nth(i)).value();
+ return MapReduceAlgebra.BSP(nn,
+ evalF(superstep,env),
+ evalE(state,env),
+ o.equals(#<true>),
+ ds);
+ case BSP(`n,`superstep,`state,`o,...as):
+ if (Config.hadoop_mode)
+ return Evaluator.evaluator.bsp(e,env);
+ Bag[] ds = new Bag[as.length()];
+ for ( int i = 0; i < ds.length; i++ )
+ ds[i] = evalM(as.nth(i),env);
+ return MapReduceAlgebra.BSP(new int[]{(int)((LongLeaf)n).value()},
+ evalF(superstep,env),
+ evalE(state,env),
+ o.equals(#<true>),
+ ds);
+ case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`num):
+ if (Config.hadoop_mode)
+ return Evaluator.evaluator.loop(e,env);
+ int limit = ((MR_int)evalE(num,env)).get();
+ Bag[] s = new Bag[vs.length()];
+ for ( int i = 0; i < vs.length(); i++ )
+ s[i] = evalM(ss.nth(i),env);
+ for ( int n = 0; n < limit; n++ ) {
+ Environment nenv = env;
+ for ( int i = 0; i < vs.length(); i ++ ) {
+ s[i].materialize();
+ nenv = new Environment(vs.nth(i).toString(),s[i],nenv);
+ };
+ for ( int i = 0; i < vs.length(); i ++ )
+ s[i] = (Bag)evalM(bs.nth(i),nenv);
+ };
+ return new Tuple(s);
+ case function(tuple(...params),`tp,`body):
+ String[] as = new String[params.length()];
+ int i = 0;
+ for ( Tree param: params )
+ match param {
+ case `bind(`v,_):
+ as[i++] = v.toString();
+ };
+ return new Lambda(evalT(as,body,env));
+ case typed(`x,_):
+ return evalE(x,env);
+ case apply(`f,`arg):
+ if (!f.is_variable())
+ return evalF(f,env).eval(evalE(arg,env));
+ MRData fnc = lookup_global_binding(f.toString());
+ if (fnc == null) {
+ String s = Plan.conf.get("mrql.global."+f);
+ if (s != null)
+ try {
+ Tree ft = Tree.parse(s);
+ TopLevel.store(f.toString(),ft);
+ fnc = evalE(ft,env);
+ new_global_binding(f.toString(),fnc);
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ };
+ MRData t = evalE(arg,env);
+ if (!(t instanceof Tuple))
+ throw new Error("Expected a tuple in function application: "+t);
+ return ((Lambda)fnc).lambda().eval(t);
+ case trace(`x):
+ MRData z = evalE(x,env);
+ System.err.println("*** "+x+": "+z);
+ return z;
+ case _:
+ try {
+ if (Config.hadoop_mode)
+ return new MR_dataset(Evaluator.evaluator.eval(e,env,"-"));
+ else return evalS(e,env);
+ } catch (Exception ex) { throw new Error(ex); }
+ };
+ throw new Error("Cannot evaluate the expression: "+e);
+ } catch (Error msg) {
+ if (!Config.trace)
+ throw new Error(msg.getMessage());
+ System.err.println(msg.getMessage());
+ msg.printStackTrace();
+ throw new Error("Evaluation error in: "+print_query(e));
+ } catch (Exception ex) {
+ if (Config.trace) {
+ System.err.println(ex.getMessage());
+ ex.printStackTrace();
+ }
+ throw new Error("Evaluation error in: "+print_query(e));
+ }
+ }
+
+ /** evaluate an MRQL expression in memory */
+ final static MRData evalE ( final Tree e ) {
+ return evalE(e,null);
+ }
+
+ /** evaluate MRQL physical operators in memory (returns a Bag) */
+ final static Bag evalS ( final Tree e, final Environment env ) {
+ return evalM(e,env);
+ }
+
+ /** evaluate MRQL physical operators in memory (returns a Bag) */
+ final static Bag evalM ( final Tree e, final Environment env ) {
+ if (Config.trace_execution) {
+ tab_count += 3;
+ System.out.println(tabs(tab_count)+print_query(e));
+ };
+ Bag res = evalMM(e,env);
+ if (Config.trace_execution) {
+ System.out.println(tabs(tab_count)+"-> "+res);
+ tab_count -= 3;
+ };
+ return res;
+ }
+
+ /** evaluate MRQL physical operators in memory (returns a Bag) */
+ final static Bag evalMM ( final Tree e, final Environment env ) {
+ try {
+ match e {
+ case cMap(`f,`s):
+ return MapReduceAlgebra.cmap(evalF(f,env),evalM(s,env));
+ case AggregateMap(`f,`acc,`zero,`s):
+ return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
+ evalM(#<cMap(`f,`s)>,env)));
+ case MapReduce(`m,`r,`s,_):
+ return MapReduceAlgebra.mapReduce(
+ evalF(m,env),
+ evalF(r,env),
+ evalM(s,env));
+ case MapAggregateReduce(`m,`r,`acc,`zero,`s,_):
+ return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
+ evalM(#<MapReduce(`m,`r,`s,false)>,env)));
+ case MapCombineReduce(`m,`c,`r,`s,_):
+ return MapReduceAlgebra.mapReduce(
+ evalF(m,env),
+ evalF(r,env),
+ evalM(s,env));
+ case MapReduce2(`mx,`my,`r,`x,`y,_):
+ return MapReduceAlgebra.mapReduce2(
+ evalF(mx,env),
+ evalF(my,env),
+ evalF(r,env),
+ evalM(x,env),
+ evalM(y,env));
+ case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,_):
+ return MapReduceAlgebra.mapReduce2(
+ evalF(mx,env),
+ evalF(my,env),
+ evalF(r,env),
+ evalM(x,env),
+ evalM(y,env));
+ case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,_):
+ return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
+ evalM(#< MapReduce2(`mx,`my,`r,`x,`y,false)>,env)));
+ case MapJoin(`kx,`ky,`r,`x,`y):
+ return MapReduceAlgebra.mapJoin(
+ evalF(kx,env),
+ evalF(ky,env),
+ evalF(r,env),
+ evalM(x,env),
+ evalM(y,env));
+ case MapAggregateJoin(`kx,`ky,`r,`acc,`zero,`x,`y):
+ return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
+ evalM(#<MapJoin(`kx,`ky,`r,`x,`y)>,env)));
+ case GroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
+ return MapReduceAlgebra.groupByJoin(
+ evalF(kx,env),
+ evalF(ky,env),
+ evalF(gx,env),
+ evalF(gy,env),
+ evalF(m,env),
+ evalF(c,env),
+ evalF(r,env),
+ evalM(x,env),
+ evalM(y,env));
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return MapReduceAlgebra.crossProduct(
+ evalF(mx,env),
+ evalF(my,env),
+ evalF(r,env),
+ evalM(x,env),
+ evalM(y,env));
+ case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
+ return new Bag(MapReduceAlgebra.aggregate(evalF(acc,env),evalE(zero,env),
+ evalM(#<CrossProduct(`mx,`my,`r,`x,`y)>,env)));
+ case BinarySource(`file,_):
+ return (Bag)MapReduceAlgebra.read_binary(file.stringValue());
+ case BSPSource(`n,BinarySource(`file,_)):
+ return (Bag)MapReduceAlgebra.read_binary((int)((LongLeaf)n).value(),
+ file.stringValue());
+ case BSPSource(`n,ParsedSource(`parser,`file,...args)):
+ if (!(n instanceof LongLeaf))
+ fail;
+ Parser p = DataSource.parserDirectory.get(parser.toString()).newInstance();
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ return MapReduceAlgebra.parsedSource((int)(((LongLeaf)n).value()),p,
+ ((MR_string)evalE(file,env)).get(),args);
+ case ParsedSource(`parser,`file,...args):
+ Parser p = DataSource.parserDirectory.get(parser.toString()).newInstance();
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ return MapReduceAlgebra.parsedSource(p,((MR_string)evalE(file,env)).get(),args);
+ case Merge(`x,`y):
+ return evalM(x,env).union(evalM(y,env));
+ case Repeat(lambda(`v,`b),`s,`n):
+ final String vs = v.toString();
+ final Tree body = b;
+ Function loop = new Function() {
+ final public MRData eval ( final MRData x ) {
+ return evalM(body,new Environment(vs,x,env));
+ }
+ };
+ return MapReduceAlgebra.repeat(loop,(Bag)evalM(s,env),((MR_int)evalE(n,env)).get());
+ case Closure(lambda(`v,`b),`s,`n):
+ final String vs = v.toString();
+ final Tree body = b;
+ Function loop = new Function() {
+ final public MRData eval ( final MRData x ) {
+ return evalM(body,new Environment(vs,x,env));
+ }
+ };
+ return MapReduceAlgebra.closure(loop,(Bag)evalM(s,env),((MR_int)evalE(n,env)).get());
+ case Generator(`min,`max,`size):
+ return MapReduceAlgebra.generator(((MR_long)evalE(min,env)).get(),
+ ((MR_long)evalE(max,env)).get());
+ case BSPSource(`n,Generator(`min,`max,`size)):
+ return MapReduceAlgebra.generator((int)((LongLeaf)n).value(),
+ ((MR_long)evalE(min,env)).get(),
+ ((MR_long)evalE(max,env)).get());
+ case Dump(`s):
+ Bag bs = (Bag)evalE(s,env);
+ final Iterator<MRData> iter = bs.iterator();
+ return new Bag(new BagIterator() {
+ public boolean hasNext () {
+ return iter.hasNext();
+ }
+ public MRData next () {
+ return new Tuple(new MR_int(0),iter.next());
+ }
+ });
+ case let(`v,`u,`body):
+ return evalM(body,new Environment(v.toString(),evalE(u,env),env));
+ case apply(`f,`arg):
+ if (!f.is_variable())
+ return (Bag)evalF(f,env).eval(evalE(arg));
+ MRData fnc = lookup_global_binding(f.toString());
+ if (fnc == null)
+ throw new Error("Unknown function: "+f);
+ MRData t = evalE(arg,env);
+ if (!(t instanceof Tuple))
+ throw new Error("Expected a tuple in function application: "+t);
+ return (Bag)((Lambda)fnc).lambda().eval(t);
+ case BSPSource(`n,`s):
+ final MR_int i = new MR_int((int)((LongLeaf)n).value());
+ Bag bs = (Bag)evalE(s,env);
+ final Iterator<MRData> iter = bs.iterator();
+ return new Bag(new BagIterator() {
+ public boolean hasNext () {
+ return iter.hasNext();
+ }
+ public MRData next () {
+ return new Tuple(i,iter.next());
+ }
+ });
+ case BSP(...):
+ MRData res = evalE(e,env);
+ if (res instanceof Bag)
+ return (Bag)res;
+ else return new Bag(res);
+ case `v:
+ if (!v.is_variable())
+ fail;
+ MRData x = variable_lookup(v.toString(),env);
+ if (x != null)
+ return (Bag)x;
+ x = lookup_global_binding(v.toString());
+ if (x != null)
+ return (Bag)x;
+ throw new Error("Variable "+v+" is not bound");
+ };
+ throw new Error("Cannot evaluate the plan: "+e);
+ } catch (Error msg) {
+ if (!Config.trace)
+ throw new Error(msg.getMessage());
+ System.err.println(msg.getMessage());
+ msg.printStackTrace();
+ throw new Error("Evaluation error in: "+print_query(e));
+ } catch (Exception ex) {
+ if (Config.trace)
+ ex.printStackTrace();
+ throw new Error("Evaluation error in: "+print_query(e));
+ }
+ }
+
+ /** replace all non-free variables with their reified values */
+ private final static Tree closure ( Tree e, Environment env, Trees local_vars ) {
+ match e {
+ case lambda(`x,`b):
+ return #<lambda(`x,`(closure(b,env,local_vars.cons(x))))>;
+ case apply(`f,...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(closure(a,env,local_vars));
+ return #<apply(`f,...bs)>;
+ case `f(...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(closure(a,env,local_vars));
+ return #<`f(...bs)>;
+ case null: return null;
+ case `v:
+ if (!v.is_variable())
+ fail;
+ if (local_vars.member(v))
+ fail;
+ MRData x = variable_lookup(v.toString(),env);
+ if (x != null)
+ if (!(x instanceof MR_dataset))
+ return reify(x);
+ x = lookup_global_binding(v.toString());
+ if (x != null)
+ if (!(x instanceof MR_dataset))
+ return reify(x);
+ };
+ return e;
+ }
+
+ /** replace all non-free variables with their reified values */
+ final static Tree closure ( Tree e, Environment env ) {
+ return closure(e,env,#[]);
+ }
+
+ static Tree query_type;
+ static Tree query_plan;
+ static boolean is_dataset;
+
+ /** translate an MRQL expression e into a physical plan */
+ final static Tree translate_expression ( Tree e ) {
+ try {
+ if (Config.trace)
+ System.out.println("Query at line "+Main.parser.line_pos()+": "+print_query(e));
+ Tree qt = TypeInference.type_inference(e);
+ if (!Config.quiet_execution)
+ System.out.println("Query type: "+print_type(qt));
+ query_type = qt;
+ Tree ne = Normalization.remove_groupby(e);
+ if (Config.trace)
+ System.out.println("After removing group-by:\n"+ne.pretty(0));
+ ne = Simplification.rename(ne);
+ if (Config.trace)
+ System.out.println("After renaming variables:\n"+ne.pretty(0));
+ ne = Simplification.rename(Normalization.normalize_all(ne));
+ if (Config.trace)
+ System.out.println("Normalized query:\n"+ne.pretty(0));
+ type_inference(ne);
+ ne = QueryPlan.best_plan(ne);
+ if (Config.trace)
+ System.out.println("Best plan:\n"+ne.pretty(0));
+ ne = Simplification.rename(Translator.translate_select(ne));
+ if (Config.trace)
+ System.out.println("After removing select-queries:\n"+ne.pretty(0));
+ type_inference(ne);
+ ne = Simplification.simplify_all(ne);
+ if (Config.trace)
+ System.out.println("Algebra expression:\n"+ne.pretty(0));
+ Tree pt = type_inference(ne);
+ if (Config.trace)
+ System.out.println("Algebraic type: "+print_type(pt));
+ ne = AlgebraicOptimization.translate_all(ne);
+ if (Config.trace)
+ System.out.println("Translated expression:\n"+ne.pretty(0));
+ Tree et = TypeInference.type_inference(ne);
+ is_dataset = PlanGeneration.is_dataset_expr(ne);
+ if (Config.trace)
+ System.out.println("Physical plan type: "+print_type(et));
+ repeat_variables = #[];
+ ne = Simplification.simplify_all(ne);
+ Tree plan = PlanGeneration.makePlan(ne);
+ if (Config.bsp_mode) {
+ BSPTranslator.reset();
+ if (Config.trace)
+ System.out.println("Physical plan:\n"+plan.pretty(0));
+ plan = Materialization.materialize_terms(BSPTranslator.constructBSPplan(plan));
+ if (Config.trace)
+ System.out.println("BSP plan:\n"+plan.pretty(0));
+ else {
+ String splan = print_plan(plan,0,false);
+ if (!splan.equals("") && !Config.quiet_execution)
+ System.out.println("BSP plan:\n"+splan);
+ }
+ } else {
+ if (Config.hadoop_mode)
+ plan = PlanGeneration.physical_plan(plan);
+ plan = Materialization.materialize_terms(AlgebraicOptimization.common_factoring(plan));
+ if (Config.trace)
+ System.out.println("Physical plan:\n"+plan.pretty(0));
+ else {
+ String splan = print_plan(plan,0,false);
+ if (!splan.equals("") && !Config.quiet_execution)
+ System.out.println("Physical plan:\n"+splan);
+ }
+ };
+ if (Config.compile_functional_arguments)
+ plan = Compiler.compile(plan);
+ return plan;
+ } catch (Error x) {
+ if (Config.testing)
+ throw new Error(x);
+ if (!Config.trace && x.toString().endsWith("Type Error"))
+ return null;
+ if (x.getMessage() != null) // system error
+ System.err.println("*** MRQL System Error at line "+Main.parser.line_pos()+": "+x);
+ if (Config.trace)
+ x.printStackTrace(System.err);
+ return null;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Inv.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Inv.java b/core/src/main/java/org/apache/mrql/Inv.java
new file mode 100644
index 0000000..3faa7fe
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Inv.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+
+final public class Inv extends MRData {
+ private MRData value;
+
+ Inv ( MRData v ) { value = v; }
+
+ public void materializeAll () { value.materializeAll(); };
+
+ public MRData value () { return value; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.INV);
+ value.write(out);
+ }
+
+ final public static Inv read ( DataInput in ) throws IOException {
+ return new Inv(MRContainer.read(in));
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value.readFields(in);
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof Inv);
+ return -value.compareTo(((Inv)x).value);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ int n = MRContainer.compare(x,xs,xl,y,ys,yl,size);
+ size[0] += 1;
+ return -n;
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof Inv && value.equals(((Inv)x).value);
+ }
+
+ public int hashCode () { return value.hashCode(); }
+
+ public String toString () { return "inv("+value.toString()+")"; }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/JSON.cup
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/JSON.cup b/core/src/main/java/org/apache/mrql/JSON.cup
new file mode 100644
index 0000000..c045437
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/JSON.cup
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java_cup.runtime.*;
+
+terminal TRUE, FALSE, NULL, COLON, COMMA, O_BEGIN, O_END, A_BEGIN, A_END;
+
+terminal String STRING;
+terminal Long INTEGER;
+terminal Double DOUBLE;
+
+non terminal MRData top, json, pair, value;
+non terminal Bag members, elements;
+
+precedence left O_BEGIN, O_END, A_BEGIN, A_END;
+precedence left COMMA;
+precedence left COLON;
+
+start with top;
+
+top ::= json:e {: RESULT = e; :}
+ ;
+json ::= O_BEGIN O_END {: RESULT = new Union((byte)0,new Bag()); :}
+ | O_BEGIN members:m O_END {: RESULT = new Union((byte)0,m); :}
+ | A_BEGIN A_END {: RESULT = new Union((byte)1,new Bag()); :}
+ | A_BEGIN elements:m A_END {: RESULT = new Union((byte)1,m); :}
+ | O_BEGIN O_BEGIN INTEGER:n O_END O_END {: RESULT = new MR_variable((int)n.longValue()); :}
+ ;
+members ::= pair:p {: RESULT = new Bag(p); :}
+ | members:m COMMA pair:p {: RESULT = m.add_element(p); :}
+ ;
+pair ::= STRING:s COLON value:v {: RESULT = (new Tuple(2)).set(0,new MR_string(s)).set(1,v); :}
+ ;
+elements ::= value:v {: RESULT = new Bag(v); :}
+ | elements:es COMMA value:v {: RESULT = es.add_element(v); :}
+ ;
+value ::= json:e {: RESULT = e; :}
+ | STRING:s {: RESULT = new Union((byte)2,new MR_string(s)); :}
+ | INTEGER:n {: RESULT = new Union((byte)3,new MR_long(n.longValue())); :}
+ | DOUBLE:n {: RESULT = new Union((byte)4,new MR_double(n.doubleValue())); :}
+ | TRUE {: RESULT = new Union((byte)5,new MR_bool(true)); :}
+ | FALSE {: RESULT = new Union((byte)5,new MR_bool(false)); :}
+ | NULL {: RESULT = new Union((byte)6,new Tuple(0)); :}
+ ;
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/JSON.lex
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/JSON.lex b/core/src/main/java/org/apache/mrql/JSON.lex
new file mode 100644
index 0000000..c75d1bb
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/JSON.lex
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java_cup.runtime.Symbol;
+
+
+%%
+%class JSONLex
+%public
+%line
+%char
+%cup
+%eofval{
+ return symbol(jsym.EOF);
+%eofval}
+%{
+ public String text () { return yytext(); }
+
+ public int line_pos () { return yyline+1; }
+
+ public int char_pos () { return yychar; }
+
+ public static Symbol symbol ( int s ) {
+ return new Symbol(s);
+ }
+
+ public static Symbol symbol ( int s, Object o ) {
+ return new Symbol(s,o);
+ }
+
+%}
+
+INT = [+-]?[0-9]+
+DOUBLE = [+-]?[0-9]+([\.][0-9]+)?([eE][+-]?[0-9]+)?
+
+%%
+
+\"[^\"]*\" { return symbol(jsym.STRING,yytext().substring(1,yytext().length()-1)); }
+{INT} { return symbol(jsym.INTEGER,new Long(yytext())); }
+{DOUBLE} { return symbol(jsym.DOUBLE,new Double(yytext())); }
+true { return symbol(jsym.TRUE); }
+false { return symbol(jsym.FALSE); }
+null { return symbol(jsym.NULL); }
+\{ { return symbol(jsym.O_BEGIN); }
+\} { return symbol(jsym.O_END); }
+\[ { return symbol(jsym.A_BEGIN); }
+\] { return symbol(jsym.A_END); }
+\, { return symbol(jsym.COMMA); }
+\: { return symbol(jsym.COLON); }
+[ \t\f] { }
+[\r\n] { }
+. { throw new Error("Illegal character: "+yytext()); }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/JsonFormatParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/JsonFormatParser.java b/core/src/main/java/org/apache/mrql/JsonFormatParser.java
new file mode 100644
index 0000000..77aa891
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/JsonFormatParser.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.StringReader;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.DataOutputBuffer;
+import java_cup.runtime.Symbol;
+
+
+/** The JSON parser */
+public class JsonFormatParser implements Parser {
+ String[] tags; // split tags
+ JsonSplitter splitter;
+
+ public void initialize ( Trees args ) {
+ try {
+ if (args.length() > 0) {
+ if (!(args.nth(0) instanceof Node)
+ || !(((Node)args.nth(0)).name().equals("list")
+ || ((Node)args.nth(0)).name().equals("bag")))
+ throw new Error("Must provide a bag of synchronization property names to split the JSON source: "+args.nth(0));
+ Trees ts = ((Node)args.nth(0)).children();
+ if (ts.length() == 0)
+ throw new Error("Expected at least one synchronization tagname in JSON source: "+ts);
+ tags = new String[ts.length()];
+ for ( int i = 0; i < tags.length; i++ )
+ if (ts.nth(i) instanceof StringLeaf)
+ tags[i] = ((StringLeaf)(ts.nth(i))).value();
+ else throw new Error("Expected a synchronization tagname in JSON source: "+ts.nth(i));
+ }
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public void open ( String file ) {
+ try {
+ splitter = new JsonSplitter(tags,file,new DataOutputBuffer());
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public void open ( FSDataInputStream fsin, long start, long end ) {
+ try {
+ splitter = new JsonSplitter(tags,fsin,start,end,new DataOutputBuffer());
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public Tree type () { return new VariableLeaf("JSON"); }
+
+ public String slice () {
+ if (splitter.hasNext()) {
+ DataOutputBuffer b = splitter.next();
+ return new String(b.getData(),0,b.getLength());
+ } else return null;
+ }
+
+ public Bag parse ( String s ) {
+ try {
+ JSONLex scanner = new JSONLex(new StringReader(s));
+ JSONParser parser = new JSONParser(scanner);
+ Symbol sym = parser.parse();
+ return new Bag((MRData)sym.value);
+ } catch (Exception e) {
+ System.err.println(e);
+ return new Bag();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/JsonSplitter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/JsonSplitter.java b/core/src/main/java/org/apache/mrql/JsonSplitter.java
new file mode 100644
index 0000000..7017e72
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/JsonSplitter.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java_cup.runtime.Symbol;
+import java.util.Iterator;
+import java.io.*;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+
+/** Extract the JSON objects tagged by tags from a data split of the input stream (fsin) */
+final public class JsonSplitter implements Iterator<DataOutputBuffer> {
+ boolean in_memory;
+ FSDataInputStream fsin; // for HDFS processing
+ InputStream in; // for in-memory processing
+ JSONLex scanner;
+ String[] tags;
+ long start;
+ long end;
+ final DataOutputBuffer buffer;
+
+ JsonSplitter ( String[] tags, FSDataInputStream fsin, long start, long end,
+ DataOutputBuffer buffer ) {
+ in_memory = false;
+ this.tags = tags;
+ this.fsin = fsin;
+ this.end = end;
+ this.buffer = buffer;
+ try {
+ fsin.seek(start);
+ this.start = (start == 0) ? start : sync(start);
+ fsin.seek(this.start);
+ scanner = new JSONLex(fsin);
+ } catch ( IOException e ) {
+ System.err.println("*** Cannot parse the data split: "+fsin);
+ }
+ }
+
+ JsonSplitter ( String[] tags, String file, DataOutputBuffer buffer ) {
+ in_memory = true;
+ try {
+ in = new FileInputStream(file);
+ } catch ( Exception e ) {
+ throw new Error("Cannot open the file: "+file);
+ };
+ this.tags = tags;
+ this.buffer = buffer;
+ scanner = new JSONLex(in);
+ }
+
+ private long sync ( long start ) {
+ try {
+ long first_quote = -1;
+ for ( long offset = 0; ; offset++ ) {
+ char c = (char)fsin.read();
+ if (c == '\"') {
+ if (first_quote >= 0)
+ if ((char)fsin.read() == ':')
+ return start+first_quote;
+ first_quote = offset;
+ }
+ }
+ } catch (IOException ex) {
+ return (long)0;
+ }
+ }
+
+ public boolean hasNext () {
+ try {
+ if (in_memory || start+scanner.char_pos() < end)
+ if (skip())
+ return store();
+ return false;
+ } catch (Exception e) {
+ System.err.println(e);
+ return false;
+ }
+ }
+
+ public DataOutputBuffer next () {
+ return buffer;
+ }
+
+ public void remove () { }
+
+ boolean is_start_tag ( String tagname ) {
+ if (tags == null)
+ return true;
+ for (String tag: tags)
+ if (tag.contentEquals(tagname))
+ return true;
+ return false;
+ }
+
+ /** skip until the beginning of a split element */
+ boolean skip () throws IOException {
+ while (true) {
+ Symbol s = scanner.next_token();
+ if (s.sym == jsym.EOF || (!in_memory && start+scanner.char_pos() >= end))
+ return false;
+ if (s.sym == jsym.STRING && is_start_tag((String)s.value)) {
+ String tag = (String)s.value;
+ if (scanner.next_token().sym == jsym.COLON) {
+ buffer.reset();
+ buffer.write('{');
+ buffer.write('\"');
+ for ( int i = 0; i < tag.length(); i++ )
+ buffer.write(tag.charAt(i));
+ buffer.write('\"');
+ buffer.write(':');
+ return true;
+ }
+ }
+ }
+ }
+
+ /** store one split element into the buffer; may cross split boundaries */
+ boolean store () throws IOException {
+ int nest = 0;
+ while (true) {
+ Symbol s = scanner.next_token();
+ if (s.sym == jsym.EOF)
+ return false;
+ if (s.sym == jsym.O_BEGIN || s.sym == jsym.A_BEGIN)
+ nest++;
+ else if (s.sym == jsym.O_END || s.sym == jsym.A_END)
+ nest--;
+ String text = scanner.text();
+ for ( int i = 0; i < text.length(); i++ )
+ buffer.write(text.charAt(i));
+ if (nest == 0) {
+ buffer.write('}');
+ return true;
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Lambda.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Lambda.java b/core/src/main/java/org/apache/mrql/Lambda.java
new file mode 100644
index 0000000..d0cfd7a
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Lambda.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** an anonymous function encapsulated as MRData */
+final public class Lambda extends MRData {
+ private Function lambda;
+
+ public Lambda ( Function f ) { lambda = f; }
+
+ public void materializeAll () {};
+
+ public Function lambda () { return lambda; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ throw new Error("Functions are not serializable");
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ throw new Error("Functions are not serializable");
+ }
+
+ public int compareTo ( MRData x ) {
+ throw new Error("Functions cannot be compared");
+ }
+
+ public boolean equals ( Object x ) {
+ throw new Error("Functions cannot be compared");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/LineParser.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/LineParser.gen b/core/src/main/java/org/apache/mrql/LineParser.gen
new file mode 100644
index 0000000..fb1e068
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/LineParser.gen
@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.util.LineReader;
+
+
+/** A parser for line-oriented, character delimited text files (such as CVS) */
+final public class LineParser implements Parser {
+ final static int maxLineLength = 1000;
+ boolean in_memory;
+ FSDataInputStream fsin; // for HDFS processing
+ LineReader in;
+ BufferedReader buffered_in; // for in-memory processing
+ Text line;
+ long start;
+ long end;
+ long pos;
+ String delimiter;
+ Tree type;
+ byte[] types; // a vector of basic type ids (see MRContainer in MapReduceData)
+ int type_length;
+
+ static byte[] relational_record ( Tree tp ) {
+ match tp {
+ case record(...al):
+ Trees attrs = #[];
+ byte[] types = new byte[al.length()];
+ for ( int i = 0; i < types.length; i++ )
+ match al.nth(i) {
+ case bind(`v,any):
+ types[i] = -1;
+ if (attrs.member(v))
+ TypeInference.type_error(tp,"Duplicate record attribute name: "+v);
+ attrs = attrs.append(v);
+ case bind(`v,`t):
+ if (!t.is_variable())
+ fail;
+ types[i] = MRContainer.type_code(t.toString());
+ if (attrs.member(v))
+ TypeInference.type_error(tp,"Duplicate record attribute name: "+v);
+ attrs = attrs.append(v);
+ if (!MRContainer.basic_type(types[i]))
+ TypeInference.error("Expected a basic type for a relational attribute: "+t);
+ case `t: TypeInference.error("Expected a basic type for a relational attribute: "
+ +TypeInference.print_type(t));
+ };
+ return types;
+ case tuple(...al):
+ byte[] types = new byte[al.length()];
+ for ( int i = 0; i < types.length; i++ )
+ match al.nth(i) {
+ case any:
+ types[i] = -1;
+ case `t:
+ if (!t.is_variable())
+ fail;
+ types[i] = MRContainer.type_code(t.toString());
+ if (!MRContainer.basic_type(types[i]))
+ TypeInference.error("Expected a basic type for a relational attribute: "+t);
+ case `t: TypeInference.error("Expected a basic type for a relational attribute: "
+ +TypeInference.print_type(t));
+ };
+ return types;
+ };
+ TypeInference.error("Expected a relational record or a tuple type: "
+ +TypeInference.print_type(tp));
+ return null;
+ }
+
+ static Tree relational_record_type ( Tree tp ) {
+ match tp {
+ case record(...al):
+ Trees ts = #[];
+ for ( Tree a: al )
+ match a {
+ case bind(_,any): ;
+ case `t: ts = ts.append(t);
+ };
+ return #<record(...ts)>;
+ case tuple(...al):
+ Trees ts = #[];
+ for ( Tree a: al )
+ if (!a.equals(#<any>))
+ ts = ts.append(a);
+ return #<tuple(...ts)>;
+ };
+ TypeInference.error("Expected a relational record type: "
+ +TypeInference.print_type(tp));
+ return null;
+ }
+
+ public Tree type () {
+ return relational_record_type(type);
+ }
+
+ public void initialize ( Trees args ) {
+ if (Config.hadoop_mode && Plan.conf == null)
+ Plan.conf = Evaluator.evaluator.new_configuration();
+ if (args.length() != 2)
+ throw new Error("The line parser must have two arguments: "+args);
+ if (!(args.nth(0) instanceof StringLeaf))
+ throw new Error("Expected a delimiter: "+args.nth(0));
+ delimiter = ((StringLeaf)args.nth(0)).value();
+ if (delimiter.length() == 0)
+ throw new Error("Expected a delimiter with at least one character: "+delimiter);
+ type = ((Node)args.nth(1)).children().nth(0);
+ types = relational_record(type);
+ type_length = 0;
+ for ( int i = 0; i < types.length; i++ )
+ if (types[i] >= 0)
+ type_length++;
+ if (type_length < 1)
+ TypeInference.error("A relational record type must have at least one component: "
+ +TypeInference.print_type(type));
+ }
+
+ public void open ( String file ) {
+ in_memory = true;
+ try {
+ buffered_in = new BufferedReader(new InputStreamReader(new FileInputStream(file)),
+ 10000);
+ } catch ( Exception e ) {
+ throw new Error("Cannot open the file: "+file);
+ }
+ }
+
+ public void open ( FSDataInputStream fsin, long fstart, long fend ) {
+ in_memory = false;
+ this.fsin = fsin;
+ start = fstart;
+ end = fend;
+ line = new Text();
+ try {
+ if (start != 0) { // for all but the first data split, skip the first record
+ --start;
+ fsin.seek(start);
+ in = new LineReader(fsin,Plan.conf);
+ start += in.readLine(new Text(),0,(int) Math.min(Integer.MAX_VALUE,end-start));
+ } else in = new LineReader(fsin,Plan.conf);
+ pos = start;
+ } catch ( IOException e ) {
+ System.err.println("*** Cannot parse the data split: "+fsin);
+ this.start = end;
+ }
+ }
+
+ public String slice () {
+ try {
+ if (in_memory)
+ return buffered_in.readLine();
+ while (pos < end) {
+ int newSize = in.readLine(line,maxLineLength,
+ Math.max((int)Math.min(Integer.MAX_VALUE,end-pos),
+ maxLineLength));
+ if (newSize == 0)
+ return null;
+ pos += newSize;
+ if (newSize < maxLineLength)
+ return line.toString();
+ };
+ return null;
+ } catch ( Exception e ) {
+ System.err.println("*** Cannot slice the text: "+e);
+ return "";
+ }
+ }
+
+ private static MRData parse_value ( String text, byte type ) {
+ switch (type) {
+ case MRContainer.BYTE: return new MR_byte(Byte.parseByte(text));
+ case MRContainer.SHORT: return new MR_short(Short.parseShort(text));
+ case MRContainer.INT: return new MR_int(Integer.parseInt(text));
+ case MRContainer.LONG: return new MR_long(Long.parseLong(text));
+ case MRContainer.FLOAT: return new MR_float(Float.parseFloat(text));
+ case MRContainer.DOUBLE: return new MR_double(Double.parseDouble(text));
+ case MRContainer.CHAR: return new MR_char(text.charAt(0));
+ case MRContainer.STRING: return new MR_string(text);
+ };
+ System.err.println("*** Cannot parse the type "+MRContainer.type_names[type]+" in '"+text+"'");
+ return null;
+ }
+
+ public Bag parse ( String line ) {
+ try {
+ if (line == null)
+ return new Bag();
+ Tuple t = new Tuple(type_length);
+ int loc = 0;
+ int j = 0;
+ for ( int i = 0; i < types.length; i++ ) {
+ int k = line.indexOf(delimiter,loc);
+ if (types[i] >= 0) {
+ String s = (k > 0) ? line.substring(loc,k) : line.substring(loc);
+ MRData v = parse_value(s,types[i]);
+ if (v == null)
+ return new Bag();
+ t.set(j++,v);
+ };
+ loc = k+delimiter.length();
+ if (k < 0 && i+1 < types.length) {
+ System.err.println("*** Incomplete parsed text line: "+line);
+ return new Bag();
+ }
+ };
+ return new Bag(t);
+ } catch ( Exception e ) {
+ System.err.println("*** Cannot parse the text line: "+line);
+ return new Bag();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MRContainer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MRContainer.java b/core/src/main/java/org/apache/mrql/MRContainer.java
new file mode 100644
index 0000000..d72c452
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MRContainer.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.io.WritableComparable;
+
+
+/** A container for MRData that implements read (the deserializer) */
+final public class MRContainer implements WritableComparable<MRContainer>, Serializable {
+ transient MRData data;
+
+ public final static byte BOOLEAN = 0, BYTE = 1, SHORT = 2, INT = 3, LONG = 4,
+ FLOAT = 5, DOUBLE = 6, CHAR = 7, STRING = 8, PAIR = 9, TUPLE = 10, BAG = 11,
+ LAZY_BAG = 12, END_OF_LAZY_BAG = 13, UNION = 14, INV = 15, LAMBDA = 16,
+ VARIABLE = 17, TRIPLE = 18, NULL = 19, DATASET = 20, SYNC = 99, MORE_BSP_STEPS = 98;
+
+ public final static byte[] type_codes
+ = { BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, CHAR, STRING, NULL, PAIR, TRIPLE,
+ TUPLE, BAG, LAZY_BAG, END_OF_LAZY_BAG, UNION, INV, LAMBDA, VARIABLE, SYNC };
+
+ public final static String[] type_names
+ = { "boolean", "byte", "short", "int", "long", "float", "double", "char", "string",
+ "null", "pair", "triple", "tuple", "bag", "lazy_bag", "end_of_lazy_bag", "union",
+ "inv", "lambda", "variable", "sync", "more_bsp_steps" };
+
+ public static byte type_code ( String type_name ) {
+ for ( byte i = 0; i < type_names.length; i ++ )
+ if (type_names[i].equals(type_name))
+ return type_codes[i];
+ return -1;
+ }
+
+ public static boolean basic_type ( byte type_code ) {
+ return type_code >= 0 && type_code <= 8;
+ }
+
+ MRContainer ( MRData d ) { data = d; }
+
+ MRContainer () { data = null; }
+
+ public final static MRData end_of_lazy_bag = new MR_EOLB();
+
+ MRData data () { return data; }
+
+ public void set ( MRData v ) { data = v; }
+ final public void write ( DataOutput out ) throws IOException { data.write(out); }
+ public void readFields ( DataInput in ) throws IOException { data = read(in); }
+ public int compareTo ( MRContainer x ) { return data.compareTo(x.data); }
+ public boolean equals ( Object x ) { return data.equals(x); }
+ public int hashCode () { return data.hashCode(); }
+ public String toString () { return data.toString(); }
+
+ final public static MRData read ( DataInput in ) throws IOException {
+ final byte tag = in.readByte();
+ switch (tag) {
+ case TUPLE: return Tuple.read(in);
+ case NULL: return new Tuple(0);
+ case PAIR: return Tuple.read2(in);
+ case TRIPLE: return Tuple.read3(in);
+ case BAG: return Bag.read(in);
+ case LAZY_BAG: return Bag.lazy_read(in);
+ case END_OF_LAZY_BAG: return end_of_lazy_bag;
+ case UNION: return Union.read(in);
+ case INV: return Inv.read(in);
+ case BOOLEAN: return MR_bool.read(in);
+ case BYTE: return MR_byte.read(in);
+ case SHORT: return MR_short.read(in);
+ case INT: return MR_int.read(in);
+ case LONG: return MR_long.read(in);
+ case FLOAT: return MR_float.read(in);
+ case DOUBLE: return MR_double.read(in);
+ case CHAR: return MR_char.read(in);
+ case STRING: return MR_string.read(in);
+ case SYNC: return new MR_sync();
+ case MORE_BSP_STEPS: return new MR_more_bsp_steps();
+ };
+ throw new Error("Unrecognized MRQL type tag: "+tag);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ if (x[xs] != y[ys])
+ return x[xs] - y[ys];
+ switch (x[xs]) {
+ case TUPLE: return Tuple.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case NULL: return 0;
+ case PAIR: return Tuple.compare2(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case TRIPLE: return Tuple.compare3(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case BAG: return Bag.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case UNION: return Union.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case INV: return Inv.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case BOOLEAN: return MR_bool.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case BYTE: return MR_byte.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case SHORT: return MR_short.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case INT: return MR_int.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case LONG: return MR_long.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case FLOAT: return MR_float.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case DOUBLE: return MR_double.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case CHAR: return MR_char.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case STRING: return MR_string.compare(x,xs+1,xl-1,y,ys+1,yl-1,size);
+ case SYNC: return 0;
+ case MORE_BSP_STEPS: return 0;
+ };
+ throw new Error("Unrecognized MRQL type tag: "+x[xs]);
+ }
+
+ private void writeObject ( ObjectOutputStream out ) throws IOException {
+ data.write(out);
+ }
+
+ private void readObject ( ObjectInputStream in ) throws IOException, ClassNotFoundException {
+ data = read(in);
+ }
+
+ private void readObjectNoData () throws ObjectStreamException {}
+
+ final static class MR_EOLB extends MRData {
+ MR_EOLB () {}
+
+ public void materializeAll () {};
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.END_OF_LAZY_BAG);
+ }
+
+ public void readFields ( DataInput in ) throws IOException {}
+
+ public int compareTo ( MRData x ) { return 0; }
+
+ public boolean equals ( Object x ) { return x instanceof MR_EOLB; }
+
+ public int hashCode () { return 0; }
+
+ public String toString () {
+ return "end_of_lazy_bag";
+ }
+ }
+}
[25/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/BSPPlan.java
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/BSPPlan.java b/bsp/src/main/java/org/apache/mrql/BSPPlan.java
new file mode 100644
index 0000000..bca8ab9
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/BSPPlan.java
@@ -0,0 +1,507 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.Arrays;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hama.bsp.*;
+import org.apache.hama.bsp.sync.SyncException;
+import org.apache.hama.HamaConfiguration;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** Evaluate a BSP plan using Hama */
+final public class BSPPlan extends Plan {
+
+ final static Configuration getConfiguration ( BSPJob job ) {
+ return job.getConf(); // use job.getConfiguration() for Hama 0.6.0
+ }
+
+ /** The BSP evaluator */
+ final static class BSPop extends BSP<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> {
+ final static MRContainer null_key = new MRContainer(new MR_byte(0));
+ // special message for sub-sync()
+ final static MRData more_to_come = new MR_sync();
+ final static MRData more_supersteps = new MR_more_bsp_steps();
+
+ private int source_num;
+ private Function superstep_fnc; // superstep function
+ private MRData state; // BSP state
+ private boolean orderp; // will output be ordered?
+ private MRData source; // BSP input
+ private Function acc_fnc; // aggregator
+ private MRData acc_result; // aggregation result
+ private static String[] all_peer_names; // all BSP peer names
+ private static BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer>[] all_peers; // all BSP peers
+ // a master peer that coordinates and collects results of partial aggregations
+ private String masterTask;
+ // buffer for received messages -- regularly in a vector, but can be spilled in a local file
+ Bag msg_cache;
+ // the cache that holds all local data in memory
+ Tuple local_cache;
+
+ private static BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> getPeer ( String name ) {
+ for ( int i = 0; i < all_peer_names.length; i++ )
+ if (all_peer_names[i].equals(name))
+ return all_peers[i];
+ throw new Error("Unknown peer: "+name);
+ }
+
+ private static void setPeer ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer ) {
+ String name = peer.getPeerName();
+ for ( int i = 0; i < all_peer_names.length; i++ )
+ if (all_peer_names[i].equals(name))
+ all_peers[i] = peer;
+ }
+
+ /** shuffle values to BSP peers based on uniform hashing on key */
+ private static String shuffle ( MRData key ) {
+ return all_peer_names[Math.abs(key.hashCode()) % all_peer_names.length];
+ }
+
+ /** to exit a BSP loop, all peers must agree to exit (this is used in BSPTranslate.bspSimplify) */
+ public static MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
+ return synchronize(getPeer(peerName.get()),mr_exit);
+ }
+
+ /** to exit a BSP loop, all peers must agree to exit (this is used in BSPTranslate.bspSimplify) */
+ public static MR_bool synchronize ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer, MR_bool mr_exit ) {
+ if (!Config.hadoop_mode)
+ return mr_exit;
+ // shortcut: if we know for sure that all peers want to exit/continue, we don't have to poll
+ if (mr_exit == SystemFunctions.bsp_true_value // must be ==, not equals
+ || mr_exit == SystemFunctions.bsp_false_value)
+ return mr_exit;
+ try {
+ // this case is only used for checking the exit condition of repeat/closure
+ boolean exit = mr_exit.get();
+ if (all_peer_names.length <= 1)
+ return (exit) ? SystemFunctions.bsp_true_value : SystemFunctions.bsp_false_value;
+ if (!exit)
+ // this peer is not ready to exit, so no peer should exit
+ for ( String p: peer.getAllPeerNames() )
+ peer.send(p,new MRContainer(more_supersteps));
+ peer.sync();
+ // now exit is true if no peer sent a "more_supersteps" message
+ exit = peer.getNumCurrentMessages() == 0;
+ peer.clear();
+ return (exit) ? SystemFunctions.bsp_true_value : SystemFunctions.bsp_false_value;
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+
+ /** collect a bag from all peers by distributing the local copy s */
+ public static Bag distribute ( MR_string peerName, Bag s ) {
+ BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer = getPeer(peerName.get());
+ if (!Config.hadoop_mode)
+ return s;
+ try {
+ for ( MRData e: s )
+ for ( String p: all_peer_names )
+ peer.send(p,new MRContainer(e));
+ peer.sync();
+ MRContainer msg;
+ Bag res = new Bag();
+ while ((msg = peer.getCurrentMessage()) != null)
+ if (!res.contains(msg.data()))
+ res.add(msg.data());
+ peer.clear();
+ return res;
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+
+ private void readLocalSources ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
+ throws IOException {
+ MRContainer key = new MRContainer();
+ MRContainer value = new MRContainer();
+ while (peer.readNext(key,value)) {
+ Tuple p = (Tuple)(value.data());
+ ((Bag)local_cache.get(((MR_int)p.first()).get())).add(p.second());
+ }
+ }
+
+ private void writeLocalResult ( Bag result,
+ BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
+ throws IOException {
+ MRContainer key = new MRContainer();
+ MRContainer value = new MRContainer();
+ for ( MRData v: result )
+ if (orderp) { // prepare for sorting
+ Tuple t = (Tuple)v;
+ key.set(t.get(1));
+ value.set(t.get(0));
+ peer.write(key,value);
+ } else {
+ value.set(v);
+ peer.write(null_key,value);
+ }
+ }
+
+ /** receive messages from other peers */
+ private void receive_messages ( final BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
+ throws IOException, SyncException, InterruptedException {
+ if (Config.bsp_msg_size <= 0) { // no buffering
+ msg_cache = new Bag(new BagIterator() {
+ MRContainer msg;
+ public boolean hasNext () {
+ try {
+ return (msg = peer.getCurrentMessage()) != null;
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+ public MRData next () {
+ return msg.data();
+ }
+ });
+ } else {
+ boolean expect_more = false; // are we expecting more incoming messages?
+ do {
+ // just in case this peer did a regular-sync() before the others did a sub-sync()
+ expect_more = false;
+ MRContainer msg;
+ // cache the received messages
+ while ((msg = peer.getCurrentMessage()) != null)
+ // if at least one peer sends a more_to_come message, then expect_more
+ if (msg.data().equals(more_to_come))
+ expect_more = true;
+ else msg_cache.add(msg.data());
+ if (expect_more)
+ peer.sync(); // sub-sync()
+ } while (expect_more);
+ }
+ }
+
+ /** send the messages produced by a superstep to peers and then receive the replies */
+ private void send_messages ( Bag msgs,
+ BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
+ throws IOException, SyncException, InterruptedException {
+ int size = 0;
+ if (Config.bsp_msg_size > 0)
+ msg_cache.clear();
+ for ( MRData m: msgs ) {
+ Tuple t = (Tuple)m;
+ // if there are too many messages to send, then sub-sync()
+ if ( Config.bsp_msg_size > 0 && size++ > Config.bsp_msg_size ) {
+ // tell all peers that there is more to come after sync
+ for ( String p: all_peer_names )
+ if (!peer.getPeerName().equals(p))
+ peer.send(p,new MRContainer(more_to_come));
+ peer.sync(); // sub-sync()
+ size = 0;
+ MRContainer msg;
+ // cache the received messages
+ while ((msg = peer.getCurrentMessage()) != null)
+ if (!msg.data().equals(more_to_come))
+ msg_cache.add(msg.data());
+ };
+ // suffle messages based on key; needs new MRContainer object
+ peer.send(shuffle(t.get(0)),new MRContainer(t.get(1)));
+ };
+ peer.sync(); // regular-sync()
+ receive_messages(peer);
+ }
+
+ @Override
+ public void bsp ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
+ throws IOException, SyncException, InterruptedException {
+ final Tuple stepin = new Tuple(4);
+ stepin.set(3,new MR_string(peer.getPeerName()));
+ Tuple result;
+ boolean skip = false;
+ String tabs = "";
+ int step = 0;
+ boolean exit;
+ if (Evaluator.evaluator == null)
+ try {
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
+ } catch (Exception ex) {
+ throw new Error(ex);
+ };
+ readLocalSources(peer);
+ setPeer(peer);
+ do {
+ if (!skip)
+ step++;
+ if (!skip && Config.trace_execution) {
+ tabs = Interpreter.tabs(Interpreter.tab_count);
+ System.err.println(tabs+" Superstep "+step+" ["+peer.getPeerName()+"]:");
+ System.err.println(tabs+" messages ["+peer.getPeerName()+"]: "+msg_cache);
+ System.err.println(tabs+" state ["+peer.getPeerName()+"]: "+state);
+ for ( int i = 0; i < local_cache.size(); i++)
+ if (local_cache.get(i) instanceof Bag && ((Bag)local_cache.get(i)).size() > 0)
+ System.out.println(tabs+" cache ["+peer.getPeerName()+"] "+i+": "+local_cache.get(i));
+ };
+ stepin.set(0,local_cache);
+ stepin.set(1,msg_cache);
+ stepin.set(2,state);
+ // evaluate one superstep
+ result = (Tuple)superstep_fnc.eval(stepin);
+ Bag msgs = (Bag)result.get(0);
+ exit = ((MR_bool)result.get(2)).get();
+ state = result.get(1);
+ // shortcuts: if we know for sure that all peers want to exit/continue
+ if (result.get(2) == SystemFunctions.bsp_true_value) { // must be ==, not equals
+ peer.sync();
+ if (Config.trace_execution)
+ System.err.println(tabs+" result ["+peer.getPeerName()+"]: "+result);
+ break;
+ };
+ if (result.get(2) == SystemFunctions.bsp_false_value) {
+ if (Config.trace_execution)
+ System.err.println(tabs+" result ["+peer.getPeerName()+"]: "+result);
+ send_messages(msgs,peer);
+ skip = false;
+ continue;
+ };
+ // shortcut: skip is true when NONE of the peers sent any messages
+ skip = (msgs == SystemFunctions.bsp_empty_bag); // must be ==, not equals
+ if (skip)
+ continue;
+ if (Config.trace_execution)
+ System.err.println(tabs+" result ["+peer.getPeerName()+"]: "+result);
+ exit = synchronize(peer,(MR_bool)result.get(2)).get();
+ send_messages(msgs,peer);
+ } while (!exit);
+ if (acc_result == null) {
+ // the BSP result is a bag that needs to be dumped to the HDFS
+ writeLocalResult((Bag)(local_cache.get(source_num)),peer);
+ // if there more results, dump them to HDFS
+ final MR_long key = new MR_long(0);
+ final MRContainer key_container = new MRContainer(key);
+ final MRContainer data_container = new MRContainer(new MR_int(0));
+ int loc = 0;
+ while ( loc < all_peer_names.length && peer.getPeerName().equals(all_peer_names[loc]) )
+ loc++;
+ Configuration conf = peer.getConfiguration();
+ String[] out_paths = conf.get("mrql.output.paths").split(",");
+ for ( int i = 1; i < out_paths.length; i++ ) {
+ String[] s = out_paths[i].split(":");
+ int out_num = Integer.parseInt(s[0]);
+ Path path = new Path(s[1]+"/peer"+loc);
+ FileSystem fs = path.getFileSystem(conf);
+ SequenceFile.Writer writer
+ = new SequenceFile.Writer(fs,conf,path,
+ MRContainer.class,MRContainer.class);
+ long count = 0;
+ for ( MRData e: (Bag)(local_cache.get(out_num)) ) {
+ key.set(count++);
+ data_container.set(e);
+ writer.append(key_container,data_container);
+ };
+ writer.close();
+ };
+ } else {
+ // the BSP result is an aggregation:
+ // send the partial results to the master peer
+ peer.send(masterTask,new MRContainer(local_cache.get(source_num)));
+ peer.sync();
+ if (peer.getPeerName().equals(masterTask)) {
+ // only the master peer collects the partial aggregations
+ MRContainer msg;
+ while ((msg = peer.getCurrentMessage()) != null)
+ acc_result = acc_fnc.eval(new Tuple(acc_result,msg.data()));
+ // write the final aggregation result
+ peer.write(null_key,new MRContainer(acc_result));
+ }
+ }
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void setup ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer ) {
+ try {
+ super.setup(peer);
+ Configuration conf = peer.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ all_peer_names = peer.getAllPeerNames();
+ all_peers = new BSPPeerImpl[all_peer_names.length];
+ Arrays.sort(all_peer_names); // is this necessary?
+ source_num = conf.getInt("mrql.output.tag",0);
+ Tree code = Tree.parse(conf.get("mrql.superstep"));
+ superstep_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.initial.state"));
+ state = Interpreter.evalE(code);
+ if (conf.get("mrql.zero") != null && !conf.get("mrql.zero").equals("")) {
+ code = Tree.parse(conf.get("mrql.zero"));
+ acc_result = Interpreter.evalE(code);
+ code = Tree.parse(conf.get("mrql.accumulator"));
+ acc_fnc = functional_argument(conf,code);
+ } else acc_result = null;
+ orderp = conf.getBoolean("mrql.orderp",false);
+ masterTask = all_peer_names[peer.getNumPeers()/2];
+ msg_cache = new Bag(1000);
+ local_cache = new Tuple(max_input_files);
+ for ( int i = 0; i < max_input_files; i++ )
+ local_cache.set(i,new Bag());
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new Error("Cannot setup the Hama BSP job: "+e);
+ }
+ }
+
+ @Override
+ public void cleanup ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer ) throws IOException {
+ if (!Config.local_mode)
+ clean();
+ local_cache = null;
+ super.cleanup(peer);
+ }
+ }
+
+ /** set Hama's min split size and number of BSP tasks (doesn't work with Hama 0.6.0) */
+ public static void setupSplits ( BSPJob job, DataSet ds ) throws IOException {
+ long[] sizes = new long[ds.source.size()];
+ if (sizes.length > Config.nodes)
+ throw new Error("Cannot distribute "+sizes.length+" files over "+Config.nodes+" BSP tasks");
+ for ( int i = 0; i < sizes.length; i++ )
+ sizes[i] = ds.source.get(i).size(Plan.conf);
+ long total_size = 0;
+ for ( long size: sizes )
+ total_size += size;
+ long split_size = Math.max(total_size/Config.nodes,100000);
+ int tasks = 0;
+ do { // adjust split_size
+ tasks = 0;
+ for ( long size: sizes )
+ tasks += (int)Math.ceil(size/(double)split_size);
+ if (tasks > Config.nodes)
+ split_size = (long)Math.ceil((double)split_size*1.01);
+ } while (tasks > Config.nodes);
+ job.setNumBspTask(tasks);
+ System.err.println("*** Using "+tasks+" BSP tasks (out of a max "+Config.nodes+")."
+ +" Each task will handle about "+Math.min(total_size/Config.nodes,split_size)
+ +" bytes of input data.");
+ job.set("bsp.min.split.size",Long.toString(split_size));
+ }
+
+ /** Evaluate a BSP operation that returns a DataSet
+ * @param source_nums output tags
+ * @param superstep the superstep function
+ * @param init_state initial state
+ * @param orderp do we need to order the result?
+ * @param source input dataset
+ * @return a new data source that contains the result
+ */
+ public final static MRData BSP ( int[] source_nums, // output tags
+ Tree superstep, // superstep function
+ Tree init_state, // initial state
+ boolean orderp, // do we need to order the result?
+ DataSet source // input dataset
+ ) throws Exception {
+ String[] newpaths = new String[source_nums.length];
+ newpaths[0] = new_path(conf);
+ conf.set("mrql.output.paths",source_nums[0]+":"+newpaths[0]);
+ for ( int i = 1; i < source_nums.length; i++ ) {
+ newpaths[i] = new_path(conf);
+ Path path = new Path(newpaths[1]);
+ FileSystem fs = path.getFileSystem(conf);
+ fs.mkdirs(path);
+ conf.set("mrql.output.paths",conf.get("mrql.output.paths")+","+source_nums[i]+":"+newpaths[i]);
+ };
+ conf.set("mrql.superstep",superstep.toString());
+ conf.set("mrql.initial.state",init_state.toString());
+ conf.set("mrql.zero","");
+ conf.setInt("mrql.output.tag",source_nums[0]);
+ conf.setBoolean("mrql.orderp",orderp);
+ BSPJob job = new BSPJob((HamaConfiguration)conf,BSPop.class);
+ setupSplits(job,source);
+ job.setJobName(newpaths[0]);
+ distribute_compiled_arguments(getConfiguration(job));
+ job.setBspClass(BSPop.class);
+ Path outpath = new Path(newpaths[0]);
+ job.setOutputPath(outpath);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setOutputFormat(SequenceFileOutputFormat.class);
+ job.setInputFormat(MultipleBSPInput.class);
+ FileInputFormat.setInputPaths(job,source.merge());
+ job.waitForCompletion(true);
+ if (source_nums.length == 1) {
+ BinaryDataSource ds = new BinaryDataSource(source_nums[0],newpaths[0],conf);
+ ds.to_be_merged = orderp;
+ return new MR_dataset(new DataSet(ds,0,3));
+ } else {
+ MRData[] s = new MRData[source_nums.length];
+ for ( int i = 0; i < source_nums.length; i++ ) {
+ BinaryDataSource ds = new BinaryDataSource(source_nums[i],newpaths[i],conf);
+ ds.to_be_merged = orderp;
+ s[i] = new MR_dataset(new DataSet(ds,0,3));
+ };
+ return new Tuple(s);
+ }
+ }
+
+ /** Evaluate a BSP operation that aggregates the results
+ * @param source_num output tag
+ * @param superstep the superstep function
+ * @param init_state initial state
+ * @param acc_fnc accumulator function
+ * @param zero zero value for the accumulator
+ * @param source input dataset
+ * @return the aggregation result
+ */
+ public final static MRData BSPaggregate ( int source_num, // output tag
+ Tree superstep, // superstep function
+ Tree init_state, // initial state
+ Tree acc_fnc, // accumulator function
+ Tree zero, // zero value for the accumulator
+ DataSet source // input dataset
+ ) throws Exception {
+ String newpath = new_path(conf);
+ conf.set("mrql.superstep",superstep.toString());
+ conf.set("mrql.initial.state",init_state.toString());
+ conf.set("mrql.accumulator",acc_fnc.toString());
+ conf.set("mrql.zero",zero.toString());
+ conf.setInt("mrql.output.tag",source_num);
+ conf.setBoolean("mrql.orderp",false);
+ BSPJob job = new BSPJob((HamaConfiguration)conf,BSPop.class);
+ setupSplits(job,source);
+ job.setJobName(newpath);
+ distribute_compiled_arguments(getConfiguration(job));
+ job.setBspClass(BSPop.class);
+ Path outpath = new Path(newpath);
+ job.setOutputPath(outpath);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setOutputFormat(SequenceFileOutputFormat.class);
+ job.setInputFormat(MultipleBSPInput.class);
+ FileInputFormat.setInputPaths(job,source.merge());
+ job.waitForCompletion(true);
+ FileSystem fs = outpath.getFileSystem(conf);
+ FileStatus[] files = fs.listStatus(outpath);
+ for ( int i = 0; i < files.length; i++ ) {
+ SequenceFile.Reader sreader = new SequenceFile.Reader(fs,files[i].getPath(),conf);
+ MRContainer key = new MRContainer();
+ MRContainer value = new MRContainer();
+ sreader.next(key,value);
+ sreader.close();
+ if (value.data() != null)
+ return value.data();
+ };
+ return null;
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/MultipleBSPInput.java
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/MultipleBSPInput.java b/bsp/src/main/java/org/apache/mrql/MultipleBSPInput.java
new file mode 100644
index 0000000..aa43423
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/MultipleBSPInput.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.Trees;
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hama.bsp.*;
+
+
+/** A FileInputFormat for multiple files, where each file may be associated with
+ * a different FileInputFormat */
+final public class MultipleBSPInput extends BSPMRQLFileInputFormat {
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split, BSPJob job ) throws IOException {
+ if (Evaluator.evaluator == null)
+ try {
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
+ } catch (Exception ex) {
+ throw new Error(ex);
+ };
+ String path = ((FileSplit)split).getPath().toString();
+ Configuration conf = BSPPlan.getConfiguration(job);
+ DataSource ds = DataSource.get(path,conf);
+ Plan.conf = conf;
+ if (ds instanceof ParsedDataSource)
+ return new BSPParsedInputFormat.ParsedRecordReader((FileSplit)split,
+ job,
+ ((ParsedDataSource)ds).parser,
+ ds.source_num,
+ (Trees)((ParsedDataSource)ds).args);
+ else if (ds instanceof BinaryDataSource)
+ return new BSPBinaryInputFormat.BinaryInputRecordReader((FileSplit)split,job,ds.source_num);
+ else if (ds instanceof GeneratorDataSource)
+ return new BSPGeneratorInputFormat.GeneratorRecordReader((FileSplit)split,ds.source_num,job);
+ else throw new Error("Unknown data source: "+ds+" for path "+path);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 097fe50..f9d447e 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -42,22 +42,6 @@
<build>
<plugins>
<plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>build-helper-maven-plugin</artifactId>
- <version>1.8</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals><goal>add-source</goal></goals>
- <configuration>
- <sources>
- <source>../src/main/java/core</source>
- </sources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<version>2.1</version>
@@ -101,8 +85,8 @@
</goals>
<configuration>
<lexDefinitions>
- <lexDefinition>../src/main/java/core/mrql.lex</lexDefinition>
- <lexDefinition>../src/main/java/core/JSON.lex</lexDefinition>
+ <lexDefinition>src/main/java/org/apache/mrql/mrql.lex</lexDefinition>
+ <lexDefinition>src/main/java/org/apache/mrql/JSON.lex</lexDefinition>
</lexDefinitions>
<outputDirectory>${project.build.directory}/generated-sources</outputDirectory>
<verbose>false</verbose>
@@ -125,7 +109,7 @@
<target>
<mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
<property name="compile_classpath" refid="maven.compile.classpath" />
- <fileset id="gen.path" dir="../src/main/java/core" includes="*.gen" />
+ <fileset id="gen.path" dir="src/main/java/org/apache/mrql" includes="*.gen" />
<pathconvert pathsep=" " property="gen.files" refid="gen.path" />
<java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
<arg line="${gen.files}" />
@@ -133,13 +117,13 @@
<arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
</java>
<java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
- <arg file="../src/main/java/core/mrql.cgen" />
+ <arg file="src/main/java/org/apache/mrql/mrql.cgen" />
<arg line="-o" />
<arg file="${project.build.directory}/generated-sources/mrql.cup" />
</java>
<taskdef name="cup" classname="java_cup.anttask.CUPTask" classpathref="maven.compile.classpath" />
<cup srcfile="${project.build.directory}/generated-sources/mrql.cup" destdir="${project.build.directory}/generated-sources" nosummary="true" parser="MRQLParser" />
- <cup srcfile="src/main/java/core/JSON.cup" destdir="${project.build.directory}/generated-sources" nosummary="true" parser="JSONParser" symbols="jsym" />
+ <cup srcfile="${project.basedir}/src/main/java/org/apache/mrql/JSON.cup" destdir="${project.build.directory}/generated-sources" nosummary="true" parser="JSONParser" symbols="jsym" />
</target>
</configuration>
</execution>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/AlgebraicOptimization.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/AlgebraicOptimization.gen b/core/src/main/java/org/apache/mrql/AlgebraicOptimization.gen
new file mode 100644
index 0000000..d574e20
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/AlgebraicOptimization.gen
@@ -0,0 +1,315 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+
+/**
+ * Optimize MRQL algebra expressions using normalization rules and heuristic optimizations
+ */
+public class AlgebraicOptimization extends Simplification {
+
+ /** Is this a group-by operation?*/
+ private static boolean is_groupBy ( Tree e ) {
+ match e {
+ case groupBy(...): return true;
+ case orderBy(...): return true;
+ };
+ return false;
+ }
+
+ /**
+ * algebraic optimization (algebra to algebra)
+ * @param e the algebraic form to be optimized
+ * @return the optimized form
+ */
+ public static Tree translate ( Tree e ) {
+ match e {
+ // if the group-by key is the same as the join key, fuse the group-by into the join
+ case mapReduce2(lambda(`v,bag(tuple(nth(`v1,0),`mx))),`my,
+ lambda(`rv,`rb),
+ `groupBy(cmap(lambda(`w,`m),`X)),`Y,`o):
+ if (!v1.equals(v) || !Config.groupJoinOpt || ! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ Tree nv = new_var();
+ Tree nr = subst(#<nth(`rv,0)>,#<cmap(lambda(`v,bag(`mx)),groupBy(nth(`rv,0)))>,rb);
+ Tree res = #<mapReduce2(lambda(`w,cmap(lambda(`nv,bag(tuple(nth(`nv,0),`nv))),`m)),
+ `my,lambda(`rv,`nr),`X,`Y,`o)>;
+ res = simplify_all(rename(res));
+ TypeInference.type_inference(res);
+ return translate(res);
+ // same for the right join input
+ case mapReduce2(`mx,lambda(`v,bag(tuple(nth(`v1,0),`my))),
+ lambda(`rv,`rb),
+ `X,`groupBy(cmap(lambda(`w,`m),`Y)),`o):
+ if (!v1.equals(v) || !Config.groupJoinOpt || ! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ Tree nv = new_var();
+ Tree nr = subst(#<nth(`rv,1)>,#<cmap(lambda(`v,bag(`my)),groupBy(nth(`rv,1)))>,rb);
+ Tree res = #<mapReduce2(`mx,lambda(`w,cmap(lambda(`nv,bag(tuple(nth(`nv,0),`nv))),`m)),
+ lambda(`rv,`nr),`X,`Y,`o)>;
+ res = simplify_all(rename(res));
+ TypeInference.type_inference(res);
+ return translate(res);
+ // convert self-joins to single-source mapreduce; Hadoop doesn't like self-joins anyway
+ case mapReduce2(lambda(`vx,`bx),lambda(`vy,`by),lambda(`s,`f),`X,`Y,`o):
+ if (!alpha_equivalent(X,Y) || !Config.selfJoinOpt)
+ fail;
+ Tree ny = subst(vy,vx,by);
+ Tree tx = null;
+ Tree ty = null;
+ match TypeInference.type_inference(bx) {
+ case _(tuple(_,`t)): tx = t;
+ };
+ match TypeInference.type_inference(by) {
+ case _(tuple(_,`t)): ty = t;
+ };
+ Tree b = subst(s,#<tuple(cmap(lambda(tuple(n,v1,v2),
+ if(call(eq,n,1),bag(v1),bag())),s),
+ cmap(lambda(tuple(n,v1,v2),
+ if(call(eq,n,2),bag(v2),bag())),s))>,
+ f);
+ Tree res = #<mapReduce(lambda(`vx,call(plus,
+ cmap(lambda(x,bag(tuple(nth(x,0),tuple(1,nth(x,1),typed(null,`ty))))),`bx),
+ cmap(lambda(y,bag(tuple(nth(y,0),tuple(2,typed(null,`tx),nth(y,1))))),`ny))),
+ lambda(tuple(k,s),`b),
+ `X,`o)>;
+ res = simplify_all(rename(res));
+ TypeInference.type_inference(res);
+ return translate(res);
+ case mapReduce2(`mx,`my,`r,cmap(lambda(`v,`b),`X),`Y,`o):
+ match X {
+ case groupBy(...): ;
+ case _: return translate(#<mapReduce2(lambda(`v,cmap(`mx,`b)),`my,`r,`X,`Y,`o)>);
+ };
+ fail
+ case mapReduce2(`mx,`my,`r,`X,cmap(lambda(`v,`b),`Y),`o):
+ match Y {
+ case groupBy(...): ;
+ case _: return translate(#<mapReduce2(`mx,lambda(`v,cmap(`my,`b)),`r,`X,`Y,`o)>);
+ };
+ fail
+ case crossProduct(`mx,`my,`r,cmap(lambda(`vx,`bx),`X),cmap(lambda(`vy,`by),`Y)):
+ return translate(#<crossProduct(lambda(`vx,cmap(`mx,`bx)),
+ lambda(`vy,cmap(`my,`by)),`r,`X,`Y)>);
+ case crossProduct(`mx,`my,`r,cmap(lambda(`v,`b),`X),`Y):
+ return translate(#<crossProduct(lambda(`v,cmap(`mx,`b)),`my,`r,`X,`Y)>);
+ case crossProduct(`mx,`my,`r,`X,cmap(lambda(`v,`b),`Y)):
+ return translate(#<crossProduct(`mx,lambda(`v,cmap(`my,`b)),`r,`X,`Y)>);
+ case cmap(`m,crossProduct(`mx,`my,lambda(`v,`b),`X,`Y)):
+ return translate(#<crossProduct(`mx,`my,lambda(`v,cmap(`m,`b)),`X,`Y)>);
+ case cmap(`r,`groupBy1(cmap(`m,`groupBy2(`s)))):
+ if (! #[groupBy,orderBy].member(#<`groupBy1>)
+ && ! #[groupBy,orderBy].member(#<`groupBy2>))
+ fail;
+ return #<mapReduce(`(identity()),
+ `(translate(r)),
+ `(translate(#<cmap(`m,`groupBy2(`s))>)),
+ `((#<`groupBy1>.equals(#<orderBy>)) ? #<true> : #<false>))>;
+ case cmap(`r,`groupBy(cmap(`m,`s))):
+ if (! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ return #<mapReduce(`(translate(m)),
+ `(translate(r)),
+ `(translate(s)),
+ `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
+ case `groupBy(cmap(`m,groupBy(`s))):
+ if (! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ return #<mapReduce(`(identity()),
+ `(identity()),
+ `(translate(#<cmap(`m,groupBy(`s))>)),
+ `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
+ case `groupBy(cmap(`m,`s)):
+ if (! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ return #<mapReduce(`(translate(m)),
+ `(identity()),
+ `(translate(s)),
+ `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
+ case cmap(`r,`groupBy(`s)):
+ if (! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ return #<mapReduce(`(identity()),
+ `(translate(r)),
+ `(translate(s)),
+ `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
+ case `groupBy(`s):
+ if (! #[groupBy,orderBy].member(#<`groupBy>))
+ fail;
+ return #<mapReduce(`(identity()),
+ `(identity()),
+ `(translate(s)),
+ `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
+ case cmap(`m,`s):
+ return #<cmap(`(translate(m)),
+ `(translate(s)))>;
+ // convert self-joins to single-source mapreduce; Hadoop doesn't like self-joins anyway
+ case join(lambda(`vx,`bx),lambda(`vy,`by),lambda(`s,`f),`x,`y):
+ if (!x.equals(y) || !Config.selfJoinOpt)
+ fail;
+ Tree ny = subst(vy,vx,by);
+ Tree b = subst(s,
+ #<tuple(cmap(lambda(tuple(n,v),
+ if(call(eq,n,1),bag(v),bag())),s),
+ cmap(lambda(tuple(n,v),
+ if(call(eq,n,2),bag(v),bag())),s))>,
+ f);
+ Tree res = #<mapReduce(lambda(`vx,bag(tuple(`bx,tuple(1,`vx)),
+ tuple(`ny,tuple(2,`vx)))),
+ lambda(tuple(k,s),`b),`x,false)>;
+ res = simplify_all(rename(res));
+ TypeInference.type_inference(res);
+ return translate(res);
+ case join(lambda(`vx,`bx),lambda(`vy,`by),`f,`x,`y):
+ return translate(#<mapReduce2(lambda(`vx,bag(tuple(`bx,`vx))),
+ lambda(`vy,bag(tuple(`by,`vy))),
+ `f,`x,`y,false)>);
+ case nth(`x,`n):
+ match TypeInference.type_inference2(x) {
+ case `S(tuple(...bl)):
+ if (!is_collection(S))
+ fail;
+ Tree nv = new_var();
+ type_env.insert(nv.toString(),bl.nth((int)n.longValue()));
+ return translate(#<cmap(lambda(`nv,`S(nth(`nv,`n))),`x)>);
+ };
+ fail
+ case project(`x,`a):
+ match TypeInference.type_inference2(x) {
+ case `S(record(...bl)):
+ if (!is_collection(S))
+ fail;
+ for ( Tree b: bl )
+ match b {
+ case bind(`c,_):
+ if (!a.equals(c))
+ fail;
+ Tree nv = new_var();
+ type_env.insert(nv.toString(),#<record(...bl)>);
+ return translate(#<cmap(lambda(`nv,`S(project(`nv,`a))),`x)>);
+ };
+ };
+ fail
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(translate(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ /** apply algebraic optimizations multiple times until no change */
+ public static Tree translate_all ( Tree e ) {
+ Tree ne = translate(e);
+ if (e.equals(ne))
+ return e;
+ else return translate_all(ne);
+ }
+
+ /**
+ * does a form contain a bulk plan that doesn't refer to a given variable?
+ * @param e the form
+ * @param v the given variable
+ * @return true if e contains a bulk plan that doesn't refer to v
+ */
+ private static boolean contains_plan ( Tree e, Tree v ) {
+ match e {
+ case lambda(`x,`u): return false;
+ case let(...): return false;
+ case Let(...): return false;
+ case `f(...as):
+ if (plan_names.member(#<`f>) && !free_variables(e,#[]).member(v))
+ return true;
+ for (Tree a: as)
+ if (contains_plan(a,v))
+ return true;
+ return false;
+ };
+ return false;
+ }
+
+ /**
+ * extract the common factors (common sub-expressions) from a form
+ * that do not refer to a given variable
+ * @param e the form
+ * @param v the given variable
+ * @return the list of common factors
+ */
+ private static Trees common_factors ( Tree e, Tree v ) {
+ match e {
+ case lambda(`x,`u): return #[];
+ case let(...): return #[];
+ case Let(...): return #[];
+ case `f(...as):
+ if (!contains_plan(e,v))
+ fail;
+ if (plan_names.member(#<`f>) && !free_variables(e,#[]).member(v))
+ return #[`e];
+ Trees bs = #[];
+ for ( Tree a: as )
+ bs = bs.append(common_factors(a,v));
+ return bs;
+ };
+ return #[];
+ }
+
+ /**
+ * if a term is used multiple times in a query, factor it out using let-expressions
+ * @param e the expression to be factored-out
+ * @return the factored-out expression
+ */
+ public static Tree common_factoring ( Tree e ) {
+ match e {
+ case `f(...as):
+ if (!plan_names.member(#<`f>))
+ fail;
+ Trees bs = #[];
+ Trees binds = #[];
+ for ( Tree a: as )
+ match a {
+ case lambda(`v,`u):
+ if (!contains_plan(u,v))
+ fail;
+ Trees gs = common_factors(u,v);
+ Tree nb = u;
+ for ( Tree g: gs) {
+ Tree nv = new_var();
+ nb = subst(g,nv,nb);
+ binds = binds.append(#<bind(`nv,`g)>);
+ };
+ bs = bs.append(#<lambda(`v,`(common_factoring(nb)))>);
+ case _: bs = bs.append(common_factoring(a));
+ };
+ Tree res = #<`f(...bs)>;
+ for ( Tree bind: binds )
+ match bind {
+ case bind(`v,`x):
+ res = #<let(`v,`x,`res)>;
+ };
+ return res;
+ case `f(...as):
+ Trees bs = #[];
+ for ( Tree a: as )
+ bs = bs.append(common_factoring(a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/BSPTranslator.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/BSPTranslator.gen b/core/src/main/java/org/apache/mrql/BSPTranslator.gen
new file mode 100644
index 0000000..a2af7a6
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/BSPTranslator.gen
@@ -0,0 +1,753 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+
+
+/** Translates physical plans to BSP plans to be executed on Hama */
+public class BSPTranslator extends TypeInference {
+ final static int orM = ClassImporter.find_method_number("or",#[bool,bool]);
+ final static int andM = ClassImporter.find_method_number("and",#[bool,bool]);
+ final static int notM = ClassImporter.find_method_number("not",#[bool]);
+ final static int eqM = ClassImporter.find_method_number("eq",#[int,int]);
+ final static int plusM = ClassImporter.find_method_number("plus",#[int,int]);
+ final static int neqM = ClassImporter.find_method_number("neq",#[int,int]);
+ final static int geqM = ClassImporter.find_method_number("geq",#[long,long]);
+ final static int gtM = ClassImporter.find_method_number("gt",#[int,int]);
+ final static int unionM = ClassImporter.find_method_number("union",#[bag(any),bag(any)]);
+ final static int countM = ClassImporter.find_method_number("count",#[bag(any)]);
+ final static int coerceM = ClassImporter.find_method_number("coerce",#[any,int]);
+
+ final static Trees planNames = plans_with_distributed_lambdas
+ .append(#[ParsedSource,Generator,BinarySource,GroupByJoin,Repeat,Closure,repeat]);
+
+ private static int source_num = 0;
+
+ public static void reset () {
+ source_num = 0;
+ }
+
+ private static Tree new_source ( Tree e ) {
+ match e {
+ case `f(...):
+ if (planNames.member(#<`f>))
+ return preprocess(e);
+ };
+ return #<BSPSource(`(source_num++),`(preprocess(e)))>;
+ }
+
+ /** add a new source num to every BSP operation */
+ private static Tree preprocess ( Tree e ) {
+ match e {
+ case lambda(`v,`b):
+ return #<lambda(`v,`(preprocess(b)))>;
+ case Aggregate(`acc,`zero,`S):
+ return #<Aggregate(`acc,`zero,
+ `(new_source(S)))>;
+ case cMap(`m,`S):
+ return #<cMap(`m,
+ `(new_source(S)))>;
+ case AggregateMap(`m,`acc,`zero,`S):
+ return #<AggregateMap(`m,`acc,`zero,
+ `(new_source(S)))>;
+ case MapReduce(`m,`r,`S,`o):
+ return #<MapReduce(`m,`r,
+ `(new_source(S)),`o)>;
+ case MapAggregateReduce(`m,`r,`acc,`zero,`S,`o):
+ return #<MapAggregateReduce(`m,`r,`acc,`zero,
+ `(new_source(S)),`o)>;
+ case MapCombineReduce(`m,`c,`r,`S,`o):
+ return #<MapCombineReduce(`m,`c,`r,
+ `(new_source(S)),`o)>;
+ case MapReduce2(`mx,`my,`r,`x,`y,`o):
+ return #<MapReduce2(`mx,`my,`r,
+ `(new_source(x)),
+ `(new_source(y)),`o)>;
+ case MapReduce2(`mx,`my,`c,`r,`x,`y,`o):
+ return #<MapCombineReduce2(`mx,`my,`c,`r,
+ `(new_source(x)),
+ `(new_source(y)),`o)>;
+ case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,`o):
+ return #<MapAggregateReduce2(`mx,`my,`r,`acc,`zero,
+ `(new_source(x)),
+ `(new_source(y)),`o)>;
+ case MapJoin(`mx,`my,`r,`x,`y):
+ return #<MapJoin(`mx,`my,`r,
+ `(new_source(x)),
+ `(new_source(y)))>;
+ case MapAggregateJoin(`mx,`my,`r,`acc,`zero,`x,`y):
+ return #<MapAggregateJoin(`mx,`my,`r,`acc,`zero,
+ `(new_source(x)),
+ `(new_source(y)))>;
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return #<CrossProduct(`mx,`my,`r,
+ `(new_source(x)),
+ `(new_source(y)))>;
+ case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
+ return #<CrossAggregateProduct(`mx,`my,`r,`acc,`zero,
+ `(new_source(x)),
+ `(new_source(y)))>;
+ case GroupByJoin(`kx,`ky,`gx,`gy,`mr,`c,`r,`x,`y,`o):
+ return #<GroupByJoin(`kx,`ky,`gx,`gy,`mr,`c,`r,
+ `(new_source(x)),
+ `(new_source(y)),`o)>;
+ case Repeat(`f,`ds,`max):
+ return #<Repeat(`(preprocess(f)),`(preprocess(ds)),`max)>;
+ case repeat(`f,`ds,`max):
+ return #<repeat(`(preprocess(f)),`(preprocess(ds)),`max)>;
+ case Closure(`f,`ds,`max):
+ return #<Closure(`(preprocess(f)),`(preprocess(ds)),`max)>;
+ case `f(...):
+ if (! #[ParsedSource,Generator,BinarySource].member(#<`f>))
+ fail;
+ return #<BSPSource(`(source_num++),`e)>;
+ case `f(...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(preprocess(a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ /** returns the source num of a BSP operation */
+ private static int source_num ( Tree e ) {
+ match e {
+ case BSP(`i,...):
+ return (int)((LongLeaf)i).value();
+ case BSPSource(`i,_):
+ return (int)((LongLeaf)i).value();
+ };
+ return -1;
+ }
+
+ private static Tree subst_getCache_num ( int n, int m, Tree e ) {
+ match e {
+ case getCache(`cache,`k):
+ if (!k.equals(#<`n>))
+ fail;
+ return #<getCache(`cache,`m)>;
+ case `f(...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(subst_getCache_num(n,m,a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ private static Tree subst_setCache_num ( int n, int m, Tree e ) {
+ match e {
+ case setCache(`cache,`k,...r):
+ if (!k.equals(#<`n>))
+ fail;
+ return #<setCache(`cache,`m,...r)>;
+ case `f(...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(subst_setCache_num(n,m,a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ private static Tree set_cache_num ( Tree e, int n ) {
+ match e {
+ case BSP(`m,...r):
+ return subst_setCache_num((int)((LongLeaf)m).value(),n,#<BSP(`n,...r)>);
+ case BSPSource(_,`x):
+ return #<BSPSource(`n,`x)>;
+ };
+ return e;
+ }
+
+ // return the source numbers of a BSP operation
+ private static Trees source_nums ( Tree e ) {
+ match e {
+ case BSP(`i,...):
+ return #[`i];
+ case Merge(`x,`y):
+ return source_nums(x).append(source_nums(y));
+ case BSPSource(`i,_):
+ return #[`i];
+ };
+ return #[];
+ }
+
+ private static Trees source_nums ( Trees s ) {
+ Trees res = #[];
+ for ( Tree x: s )
+ res = res.append(source_nums(x));
+ return res;
+ }
+
+ private static Tree getCache ( Tree var, Tree e, Tree body ) {
+ Trees ns = source_nums(e);
+ if (ns.length() > 0)
+ return subst(var,#<getCache(cache,...ns)>,body);
+ return body;
+ }
+
+ private static Tree getCache ( Tree var, Trees s, Tree body ) {
+ Trees ns = source_nums(s);
+ return subst(var,#<getCache(cache,...ns)>,body);
+ }
+
+ /** optimize a BSP plan after BSP fusion */
+ private static Tree post_simplify ( Tree e ) {
+ match e {
+ case cmap(lambda(`x,`S(`a)),`b):
+ if (is_collection(S))
+ return post_simplify(#<map(lambda(`x,`a),`b)>);
+ else fail
+ case `f(...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(post_simplify(a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ private static Tree processBSP ( Tree e ) {
+ return post_simplify(Simplification.simplify_all(Simplification.rename(e)));
+ }
+
+ private static Tree mkBSP ( Tree superstep, Tree state, Tree orderp, Tree input ) {
+ int rn = source_num++;
+ superstep = processBSP(subst(#<o_>,#<`rn>,getCache(#<i_>,input,superstep)));
+ return #<BSP(`rn,`superstep,`state,`orderp,`input)>;
+ }
+
+ private static Tree mkBSPL ( Tree superstep, Tree state, Tree orderp, Trees input ) {
+ int rn = source_num++;
+ superstep = processBSP(subst(#<o_>,#<`rn>,getCache(#<i>,input,superstep)));
+ return #<BSP(`rn,`superstep,`state,`orderp,...input)>;
+ }
+
+ private static Tree mkBSPL ( int[] ns, Tree superstep, Tree state, Tree orderp, Trees input ) {
+ superstep = processBSP(subst(#<o_>,#<`(ns[0])>,getCache(#<i_>,input,superstep)));
+ Trees s = #[];
+ for ( int n: ns )
+ s = s.append(#<`n>);
+ return #<BSP(tuple(...s),`superstep,`state,`orderp,...input)>;
+ }
+
+ private static Tree mkBSPL ( int n, Tree superstep, Tree state, Tree orderp, Trees input ) {
+ superstep = processBSP(subst(#<o_>,#<`n>,getCache(#<i_>,input,superstep)));
+ return #<BSP(`n,`superstep,`state,`orderp,...input)>;
+ }
+
+ private static Tree mkBSP2 ( Tree superstep, Tree state, Tree orderp, Tree left, Tree right ) {
+ int rn = source_num++;
+ superstep = processBSP(subst(#<o_>,#<`rn>,getCache(#<i_>,left,
+ getCache(#<j_>,right,superstep))));
+ return #<BSP(`rn,`superstep,`state,`orderp,`left,`right)>;
+ }
+
+ /** construct a BSP plan from a physical plan
+ * @param e the physical plan
+ * @return the BSP plan
+ */
+ private static Tree mr2bsp ( Tree e ) {
+ match e {
+ case Aggregate(`acc,`zero,`S):
+ return #<Aggregate(`acc,`zero,`(mr2bsp(S)))>;
+ case cMap(`m,`S):
+ return mkBSP(#<lambda(tuple(cache,ms,k,peer),
+ setCache(cache,o_,cmap(`m,i_),
+ tuple(BAG(),tuple(),TRUE())))>,
+ #<tuple()>,
+ #<false>,
+ mr2bsp(S));
+ case AggregateMap(`m,`acc,`zero,`S):
+ return mkBSP(#<lambda(tuple(cache,ms,k,peer),
+ setCache(cache,o_,aggregate(`acc,`zero,cmap(`m,i_)),
+ tuple(BAG(),tuple(),TRUE())))>,
+ #<tuple()>,
+ #<false>,
+ mr2bsp(S));
+ case MapReduce(`m,`r,`S,`o):
+ return mkBSP(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(cmap(lambda(tuple(k,c),
+ bag(tuple(k,tuple(k,c)))),
+ cmap(`m,i_)),
+ false,
+ FALSE()),
+ setCache(cache,o_,`(o.equals(#<true>) // need to sort the result?
+ ? #<cmap(lambda(tuple(k,s),
+ cmap(lambda(x,bag(tuple(x,k))),
+ apply(`r,tuple(k,s)))),
+ groupBy(ms))>
+ : #<cmap(`r,groupBy(ms))>),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ o,
+ mr2bsp(S));
+ case MapAggregateReduce(`m,`r,`acc,`zero,`S,_):
+ return mkBSP(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(cmap(lambda(tuple(k,c),
+ bag(tuple(k,tuple(k,c)))),
+ cmap(`m,i_)),
+ false,
+ FALSE()),
+ setCache(cache,o_,aggregate(`acc,`zero,cmap(`r,groupBy(ms))),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ #<false>,
+ mr2bsp(S));
+ case MapCombineReduce(`m,`c,`r,`S,`o):
+ return mkBSP(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(cmap(lambda(tuple(k,s),
+ cmap(lambda(x,bag(tuple(k,tuple(k,x)))),
+ apply(`c,tuple(k,s)))),
+ groupBy(cmap(`m,i_))),
+ false,
+ FALSE()),
+ setCache(cache,o_,`(o.equals(#<true>) // need to sort the result?
+ ? #<cmap(lambda(tuple(k,s),
+ cmap(lambda(x,bag(tuple(x,k))),
+ apply(`r,tuple(k,s)))),
+ groupBy(ms))>
+ : #<cmap(`r,groupBy(ms))>),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ o,
+ mr2bsp(S));
+ case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,`o):
+ return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(callM(union,`unionM,
+ cmap(lambda(tuple(kx,x),
+ bag(tuple(kx,tuple(kx,tuple(1,x))))),
+ cmap(`mx,i_)),
+ cmap(lambda(tuple(ky,y),
+ bag(tuple(ky,tuple(ky,tuple(2,y))))),
+ cmap(`my,j_))),
+ false,
+ FALSE()),
+ setCache(cache,o_,cmap(lambda(tuple(k,s),
+ cmap(lambda(tuple(kk,ss),
+ cmap(lambda(x,bag(tuple(kk,x))),
+ apply(`c,tuple(kk,ss)))),
+ groupBy(apply(`r,
+ tuple(cmap(lambda(tuple(kx,x),
+ if(callM(eq,`eqM,kx,1),
+ bag(x),
+ bag())),
+ s),
+ cmap(lambda(tuple(ky,y),
+ if(callM(eq,`eqM,ky,2),
+ bag(y),
+ bag())),
+ s)))))),
+ groupBy(ms)),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ o,
+ mr2bsp(x),
+ mr2bsp(y));
+ case MapReduce2(`mx,`my,`r,`x,`y,`o):
+ return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(callM(union,`unionM,
+ cmap(lambda(tuple(kx,x),
+ bag(tuple(kx,tuple(kx,tuple(1,x))))),
+ cmap(`mx,i_)),
+ cmap(lambda(tuple(ky,y),
+ bag(tuple(ky,tuple(ky,tuple(2,y))))),
+ cmap(`my,j_))),
+ false,
+ FALSE()),
+ setCache(cache,o_,cmap(lambda(tuple(k,s),
+ cmap(lambda(x,bag(`(o.equals(#<true>) ? #<tuple(x,k)> : #<x>))),
+ apply(`r,
+ tuple(cmap(lambda(tuple(kx,x),
+ if(callM(eq,`eqM,kx,1),
+ bag(x),
+ bag())),
+ s),
+ cmap(lambda(tuple(ky,y),
+ if(callM(eq,`eqM,ky,2),
+ bag(y),
+ bag())),
+ s))))),
+ groupBy(ms)),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ o,
+ mr2bsp(x),
+ mr2bsp(y));
+ case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,_):
+ return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(callM(union,`unionM,
+ cmap(lambda(tuple(kx,x),
+ bag(tuple(kx,tuple(kx,tuple(1,x))))),
+ cmap(`mx,i_)),
+ cmap(lambda(tuple(ky,y),
+ bag(tuple(ky,tuple(ky,tuple(2,y))))),
+ cmap(`my,j_))),
+ false,
+ FALSE()),
+ setCache(cache,o_,aggregate(`acc,`zero,
+ cmap(lambda(tuple(k,s),
+ apply(`r,tuple(cmap(lambda(tuple(kx,x),
+ if(callM(eq,`eqM,kx,1),
+ bag(x),
+ bag())),
+ s),
+ cmap(lambda(tuple(ky,y),
+ if(callM(eq,`eqM,ky,2),
+ bag(y),
+ bag())),
+ s)))),
+ groupBy(ms))),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ #<false>,
+ mr2bsp(x),
+ mr2bsp(y));
+ case MapJoin(`mx,`my,`r,`x,`y):
+ return mr2bsp(#<MapReduce2(`mx,`my,
+ lambda(tuple(xs,ys),cmap(lambda(x,apply(`r,tuple(x,ys))),xs)),
+ `x,`y,false)>);
+ case MapAggregateJoin(`mx,`my,`r,`acc,`zero,`x,`y):
+ return mr2bsp(#<MapAggregateReduce2(`mx,`my,
+ lambda(tuple(xs,ys),cmap(lambda(x,apply(`r,tuple(x,ys))),xs)),
+ `acc,`zero,`x,`y)>);
+ case GroupByJoin(`kx,`ky,lambda(`vx,`gx),lambda(`vy,`gy),`mr,`c,`r,`x,`y,`o):
+ int n = 8;
+ int m = 8;
+ System.err.println("*** Using a groupBy join on a "+n+"*"+m+" grid of partitions");
+ Tree xkey = #<cmap(lambda(i,bag(tuple(call(plus,call(mod,call(hash_code,`gx),`m),call(times,`m,i)),
+ tuple(1,call(plus,call(mod,call(hash_code,`gx),`m),call(times,`m,i)),`vx)))),
+ range(0,`(n-1)))>;
+ Tree ykey = #<cmap(lambda(j,bag(tuple(call(plus,call(times,call(mod,call(hash_code,`gy),`n),`m),j),
+ tuple(2,call(plus,call(times,call(mod,call(hash_code,`gy),`n),`m),j),`vy)))),
+ range(0,`(m-1)))>;
+ type_inference(xkey);
+ type_inference(ykey);
+ xkey = PlanGeneration.makePlan(xkey);
+ ykey = PlanGeneration.makePlan(ykey);
+ return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
+ if(map_step,
+ tuple(callM(union,`unionM,
+ cmap(lambda(`vx,`xkey),i_),
+ cmap(lambda(`vy,`ykey),j_)),
+ false,
+ FALSE()),
+ setCache(cache,o_,
+ mergeGroupByJoin(`kx,`ky,lambda(`vx,`gx),lambda(`vy,`gy),`mr,`c,`r,
+ cmap(lambda(tuple(kx,p,x),
+ if(callM(eq,`eqM,kx,1),bag(tuple(p,x)),bag())),ms),
+ cmap(lambda(tuple(ky,p,y),
+ if(callM(eq,`eqM,ky,2),bag(tuple(p,y)),bag())),ms),`o),
+ tuple(BAG(),false,TRUE()))))>,
+ #<true>,
+ o,
+ mr2bsp(x),
+ mr2bsp(y));
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return mkBSP(#<lambda(tuple(cache,ms,ys,peer),
+ tuple(BAG(),
+ setCache(cache,o_,
+ cmap(lambda(x,
+ cmap(lambda(y,apply(`r,tuple(x,y))),
+ cmap(`my,ys))),
+ cmap(`mx,i_)),
+ tuple()),
+ TRUE()))>,
+ #<Collect(`(mr2bsp(y)))>,
+ #<false>,
+ mr2bsp(x));
+ case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
+ return mkBSP(#<lambda(tuple(cache,ms,ys,peer),
+ tuple(BAG(),
+ setCache(cache,o_,
+ aggregate(`acc,`zero,
+ cmap(lambda(x,
+ cmap(lambda(y,apply(`r,tuple(x,y))),
+ cmap(`my,ys))),
+ cmap(`mx,i_)),
+ tuple())),
+ TRUE()))>,
+ #<Collect(`(mr2bsp(y)))>,
+ #<false>,
+ mr2bsp(x));
+ case Repeat(lambda(`v,`b),`ds,`max):
+ Tree step = bspSimplify(mr2bsp(b));
+ int step_cache_num = source_num(step);
+ match step {
+ case BSP(`n,`s,`k0,_,...as):
+ Tree ds_source = mr2bsp(ds);
+ int ds_cache_num = source_num(ds_source);
+ ds_source = set_cache_num(ds_source,step_cache_num);
+ // the initial values of all data sources
+ Trees sources = #[`ds_source];
+ Tree step_input = #<0>;
+ for ( Tree x: as )
+ match x {
+ case BSPSource(`j,`y):
+ if (y.equals(v))
+ step_input = j;
+ else sources = sources.append(x);
+ case _: sources = sources.append(x);
+ };
+ s = subst_getCache_num((int)((LongLeaf)step_input).value(),step_cache_num,s);
+ return mkBSPL(step_cache_num,
+ #<lambda(tuple(cache,ms,tuple(k,steps),peer),
+ let(tuple(ts,kk,step_end),
+ apply(`s,tuple(cache,ms,k,peer)),
+ if(step_end, // end of repeat step
+ setCache(cache,`step_cache_num,
+ map(lambda(tuple(x,bb),x),
+ getCache(cache,`step_cache_num)),
+ tuple(bag(),
+ tuple(`k0,callM(plus,`plusM,steps,1)),
+ if(callM(gt,`gtM,steps,`max),
+ TRUE(), // if # of steps <= limit, exit
+ callM(not,`notM, // ... else check the stopping condition
+ aggregate(lambda(tuple(x,y),callM(or,`orM,x,y)),
+ false,
+ map(lambda(tuple(x,bb),bb),
+ getCache(cache,`step_cache_num))))))),
+ tuple(ts,tuple(kk,steps),FALSE()))))>,
+ #<tuple(`k0,1)>,
+ #<false>,
+ sources);
+ case `x: throw new Error("Cannot compile the repeat function: "+x);
+ }
+ // when the repeat variable is in memory
+ case repeat(lambda(`v,`b),`ds,`max):
+ Tree step = bspSimplify(mr2bsp(b));
+ int step_cache_num = source_num(step);
+ step = subst(v,#<map(lambda(tuple(x,b),x),getCache(cache,`step_cache_num))>,step);
+ match step {
+ case BSP(`n,`s,`k0,_,...as):
+ // the initial values of all data sources
+ Trees sources = #[];
+ for ( Tree x: as )
+ if (!x.equals(v))
+ sources = sources.append(x);
+ Tree res = mkBSPL(#<lambda(tuple(cache,ms,tuple(k,steps,firstp,S),peer),
+ let(ignore,if(firstp,setCache(cache,`step_cache_num,S,0),0),
+ let(tuple(ts,kk,step_end),
+ apply(`s,tuple(cache,ms,k,peer)),
+ if(step_end, // end of repeat step
+ setCache(cache,o_,map(lambda(tuple(x,b),x),getCache(cache,`step_cache_num)),
+ setCache(cache,`step_cache_num,
+ distribute(peer,getCache(cache,`step_cache_num)),
+ tuple(bag(),
+ tuple(`k0,callM(plus,`plusM,steps,1),false,bag()),
+ if(callM(gt,`gtM,steps,`max),
+ TRUE(), // if # of steps > limit, exit
+ callM(not,`notM, // ... else check the stopping condition
+ aggregate(lambda(tuple(x,y),callM(or,`orM,x,y)),
+ false,
+ map(lambda(tuple(x,bb),bb),
+ getCache(cache,`step_cache_num)))))))),
+ tuple(ts,tuple(kk,steps,false,bag()),FALSE())))))>,
+ #<tuple(`k0,1,true,map(lambda(x,tuple(x,false)),`ds))>,
+ #<false>,
+ sources);
+ return #<Collect(`res)>; // must return a memory bag
+ case `x: throw new Error("Cannot compile the repeat function: "+x);
+ }
+ case Closure(lambda(`v,`b),`ds,`max):
+ Tree step = bspSimplify(mr2bsp(b));
+ int step_cache_num = source_num(step);
+ match step {
+ case BSP(`n,`s,`k0,_,...as):
+ Tree ds_source = mr2bsp(ds);
+ int ds_cache_num = source_num(ds_source);
+ ds_source = set_cache_num(ds_source,step_cache_num);
+ // the initial values of all data sources
+ Trees sources = #[`ds_source];
+ Tree step_input = #<0>;
+ for ( Tree x: as )
+ match x {
+ case BSPSource(`j,`y):
+ if (y.equals(v))
+ step_input = j;
+ else sources = sources.append(x);
+ case _: sources = sources.append(x);
+ };
+ s = subst_getCache_num((int)((LongLeaf)step_input).value(),step_cache_num,s);
+ return mkBSPL(step_cache_num,
+ #<lambda(tuple(cache,ms,tuple(k,steps,len),peer),
+ let(tuple(ts,kk,step_end),
+ apply(`s,tuple(cache,ms,k,peer)),
+ if(step_end, // end of repeat step
+ let(newLen,callM(count,`countM,getCache(cache,`step_cache_num)),
+ tuple(bag(),
+ tuple(`k0,callM(plus,`plusM,steps,1),newLen),
+ if(callM(gt,`gtM,steps,`max),
+ TRUE(), // if # of steps > limit, exit
+ // ... else check if the new size is the same as the old size
+ callM(geq,`geqM,len,newLen)))),
+ tuple(ts,tuple(kk,steps,len),FALSE()))))>,
+ #<tuple(`k0,1,callM(coerce,`coerceM,0,4))>,
+ #<false>,
+ sources);
+ case `x: throw new Error("Cannot compile the closure function: "+x);
+ }
+ case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`max):
+ Tree[] steps = new Tree[vs.length()];
+ Tree[] inits = new Tree[vs.length()];
+ int[] cache_num = new int[vs.length()];
+ Tree[] k = new Tree[vs.length()];
+ Trees sources = #[];
+ Trees all_cache = #[];
+ for ( int i = 0; i < vs.length(); i++ ) {
+ inits[i] = mr2bsp(ss.nth(i));
+ sources = sources.append(inits[i]);
+ cache_num[i] = source_num(inits[i]);
+ all_cache = all_cache.append(#<getCache(cache,`(cache_num[i]))>);
+ };
+ for ( int i = 0; i < vs.length(); i++ )
+ match bspSimplify(mr2bsp(bs.nth(i))) {
+ case BSP(`n,`s,`k0,_,...as):
+ steps[i] = subst_setCache_num((int)((LongLeaf)n).value(),cache_num[i],s);
+ k[i] = k0;
+ loop: for ( Tree x: as )
+ match x {
+ case BSPSource(`m,`w):
+ if (!vs.member(w))
+ fail;
+ for ( int j = 0; j < vs.length(); j++ )
+ if (w.equals(vs.nth(j)))
+ steps[i] = subst_getCache_num((int)((LongLeaf)m).value(),cache_num[j],steps[i]);
+ case BSPSource(`n1,`d1):
+ for ( Tree y: sources )
+ match y {
+ case BSPSource(`n2,`d2):
+ if (d1.equals(d2)) {
+ steps[i] = subst_getCache_num((int)((LongLeaf)n1).value(),
+ (int)((LongLeaf)n2).value(),steps[i]);
+ continue loop;
+ }
+ };
+ sources = sources.append(x);
+ case _: sources = sources.append(x);
+ };
+ };
+ Tree code = #<tuple(BAG(),tuple(`(k[0]),0,1),TRUE())>;
+ for ( int i = 0; i < vs.length(); i++ )
+ code = #<if(callM(eq,`eqM,i,`i),
+ let(tuple(ts,kk,step_end),
+ apply(`(steps[i]),tuple(cache,ms,k,peer)),
+ if(step_end, // end of repeat step
+ `((i+1 < vs.length())
+ ? #<tuple(bag(),tuple(`(k[i+1]),`(i+1),steps),FALSE())>
+ : #<tuple(bag(),
+ tuple(`(k[0]),0,callM(plus,`plusM,steps,1)),
+ if(callM(gt,`gtM,steps,`max), // if # of steps > limit, exit
+ TRUE(),
+ FALSE()))>),
+ tuple(ts,tuple(kk,i,steps),FALSE()))),
+ `code)>;
+ return mkBSPL(cache_num,
+ #<lambda(tuple(cache,ms,tuple(k,i,steps),peer),`code)>,
+ #<tuple(`(k[0]),0,2)>,
+ #<false>,
+ sources);
+ case `f(...as):
+ Trees bs = #[];
+ for ( Tree a: as )
+ bs = bs.append(mr2bsp(a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ /** simplify the BSP plan by fusing consequtive BSP plans */
+ private static Tree bspSimplify ( Tree e ) {
+ match e {
+ case BSP(`n,`s2,`k2,`o,...r,BSP(_,`s1,`k1,_,...s),...t):
+ Trees ys = r.append(t);
+ Trees ns = #[];
+ loop: for ( Tree x: s )
+ match x {
+ case BSPSource(`n1,`d1):
+ for ( Tree y: ys )
+ match y {
+ case BSPSource(`n2,`d2):
+ if (d1.equals(d2)) {
+ s1 = subst_getCache_num((int)((LongLeaf)n1).value(),
+ (int)((LongLeaf)n2).value(),s1);
+ continue loop;
+ }
+ };
+ ns = ns.append(x);
+ case _: ns = ns.append(x);
+ };
+ return bspSimplify(mkBSPL((int)((LongLeaf)n).value(),
+ #<lambda(tuple(cache,ms,tuple(first,k),peer),
+ if(first,
+ let(tuple(ts,kk,b),apply(`s1,tuple(cache,ms,k,peer)),
+ let(exit,
+ synchronize(peer,b), // poll all peers: do you want to exit?
+ // all peers must aggree to exit the inner BSP
+ // and proceed to the outer BSP
+ tuple(ts,tuple(callM(not,`notM,exit),
+ if(exit,`k2,kk)),FALSE()))),
+ let(tuple(ts,kk,bb),apply(`s2,tuple(cache,ms,k,peer)),
+ tuple(ts,tuple(false,kk),bb))))>,
+ #<tuple(true,`k1)>,
+ o,
+ #[...r,...ns,...t]));
+ case `f(...as):
+ Trees bs = #[];
+ for ( Tree a: as )
+ bs = bs.append(bspSimplify(a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ private static Tree post_simplify_plan ( Tree e ) {
+ match e {
+ case setCache(`cache,`a,`v,`ret):
+ return post_simplify_plan(#<setNth(`cache,`a,materialize(`v),`ret)>);
+ case getCache(`cache,`a,...as):
+ Tree z = #<nth(`cache,`a)>;
+ for ( Tree x: as )
+ z = #<callM(union,`unionM,`z,nth(`cache,`x))>;
+ return z;
+ case `f(...as):
+ Trees bs = #[];
+ for (Tree a: as)
+ bs = bs.append(post_simplify_plan(a));
+ return #<`f(...bs)>;
+ };
+ return e;
+ }
+
+ /** construct and simplify the BSP plan from a physical plan
+ * @param plan the physical plan
+ * @return the BSP plan
+ */
+ public static Tree constructBSPplan ( Tree plan ) {
+ return post_simplify_plan(bspSimplify(mr2bsp(preprocess(plan))));
+ }
+}
[20/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Printer.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Printer.gen b/core/src/main/java/org/apache/mrql/Printer.gen
new file mode 100644
index 0000000..d0d80da
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Printer.gen
@@ -0,0 +1,483 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.Iterator;
+import java.util.List;
+
+
+/** printers for types, expressions, plans, etc */
+public class Printer {
+
+ public static String print_type ( Tree tp ) {
+ match tp {
+ case tuple(...tl):
+ if (tl.is_empty())
+ return "()";
+ String s = "( "+print_type(tl.head());
+ for ( Tree t: tl.tail() )
+ s += ", "+print_type(t);
+ return s+" )";
+ case record(...tl):
+ if (tl.is_empty())
+ return "< >";
+ String s = "< ";
+ match tl.head() {
+ case bind(`a,`t):
+ s += a+": "+print_type(t);
+ };
+ for ( Tree t: tl.tail() )
+ match t {
+ case bind(`a,`at):
+ s += ", "+a+": "+print_type(at);
+ };
+ return s+" >";
+ case arrow(`itp,`otp):
+ return print_type(itp)+" -> "+print_type(otp);
+ case persistent(`t):
+ return "!"+print_type(t);
+ case `f():
+ return f+"()";
+ case persistent(`t):
+ return "!"+print_type(t);
+ case Bag(`etp):
+ return "!bag("+print_type(etp)+")";
+ case List(`etp):
+ return "!list("+print_type(etp)+")";
+ case `f(...tl):
+ String s = f+"( "+print_type(tl.head());
+ for ( Tree t: tl.tail() )
+ s += ", "+print_type(t);
+ return s+" )";
+ };
+ return tp.toString();
+ }
+
+ private static String print_query_list ( Trees el ) {
+ if (el.length() == 0)
+ return "";
+ String s = " ";
+ s += print_query(el.head());
+ for ( Tree a: el.tail() )
+ s += ", "+print_query(a);
+ return s+" ";
+ }
+
+ public static String print_query ( Tree e ) {
+ match e {
+ case select(`opt_dist,`u,from(...bl),where(`c),groupby(...gs),orderby(...os)):
+ String s = "select "+(opt_dist.equals(#<none>) ? "" : "distinct ");
+ s += print_query(u)+" from ";
+ match bl.head() {
+ case bind(`p,`d):
+ s += print_query(p)+" in "+print_query(d);
+ };
+ for ( Tree b: bl.tail() )
+ match b {
+ case bind(`p,`d):
+ s += ", "+print_query(p)+" in "+print_query(d);
+ };
+ if (!c.equals(#<true>))
+ s += " where "+print_query(c);
+ match #<groupby(...gs)> {
+ case groupby(`h,...gl):
+ s += " group by ";
+ match gl.head() {
+ case bind(`gp,`gd):
+ s += print_query(gp)+": "+print_query(gd);
+ };
+ for ( Tree g: gl.tail() )
+ match g {
+ case bind(`gp,`gd):
+ s += ", "+print_query(gp)+": "+print_query(gd);
+ };
+ if (!h.equals(#<true>))
+ s += " having "+print_query(h);
+ };
+ match #<orderby(...os)> {
+ case orderby(`l,...ol):
+ s += " order by "+print_query(ol.length() == 1 ? ol.head() : #<tuple(...ol)>);
+ if (!l.equals(#<none>))
+ s += " limit "+print_query(l);
+ };
+ return s;
+ case tuple(...el):
+ return "("+print_query_list(el)+")";
+ case record(...el):
+ String s = "< ";
+ match el.head() {
+ case bind(`v,`b):
+ s += v+": "+print_query(b);
+ };
+ for ( Tree a: el.tail() )
+ match a {
+ case bind(`v,`b):
+ s += ", "+v+": "+print_query(b);
+ };
+ return s+" >";
+ case project(`a,`v):
+ return (a.is_variable()) ? print_query(a)+"."+v : "("+print_query(a)+")."+v;
+ case index(`a,`i):
+ return (a.is_variable()) ? print_query(a)+"["+print_query(i)+"]"
+ : "("+print_query(a)+")["+print_query(i)+"]";
+ case nth(`x,`n):
+ return (x.is_variable()) ? print_query(x)+"#"+print_query(n)
+ : "("+print_query(x)+")#"+print_query(n);
+ case call(`f,...el):
+ return f+"("+print_query_list(el)+")";
+ case list(...el):
+ return "["+print_query_list(el)+"]";
+ case bag(...el):
+ return "{"+print_query_list(el)+"}";
+ case `f(...el):
+ return f+"("+print_query_list(el)+")";
+ };
+ return e.toString();
+ }
+
+ private static String print_XML ( final Union x ) {
+ if (x.tag() == 1)
+ return ((MR_string)x.value()).get();
+ Tuple t = (Tuple)x.value();
+ String s = "<"+((MR_string)t.get(0)).get();
+ for ( MRData a: (Bag)t.get(1) ) {
+ Tuple attr = (Tuple)a;
+ s += " "+((MR_string)attr.first()).get()+"=\""
+ +((MR_string)attr.second()).get()+"\"";
+ };
+ Bag c = (Bag)t.get(2);
+ if (c.size() == 0)
+ return s+"/>";
+ s += ">";
+ for ( MRData e: c )
+ s += print_XML((Union)e);
+ return s+"</"+((MR_string)t.get(0)).get()+">";
+ }
+
+ private static String print_JSON ( final Union x ) {
+ switch (x.tag()) {
+ case 0:
+ String s = "{ ";
+ for ( MRData e: (Bag)x.value() ) {
+ Tuple t = (Tuple)e;
+ s += t.get(0)+": "+print_JSON((Union)t.get(1))+", ";
+ };
+ return s.substring(0,s.length()-2)+" }";
+ case 1:
+ String q = "[ ";
+ for ( MRData e: (Bag)x.value() )
+ q += print_JSON((Union)e)+", ";
+ return q.substring(0,q.length()-2)+" ]";
+ };
+ return ""+x.value();
+ }
+
+ /** An MRData printer based on type information */
+ final static String print ( final MRData x, final Tree type ) {
+ try {
+ if (x instanceof Inv)
+ return print(((Inv)x).value(),type);
+ if (type.equals(#<XML>))
+ return print_XML((Union)x);
+ if (type.equals(#<JSON>))
+ return print_JSON((Union)x);
+ match TypeInference.expand(type) {
+ case persistent(`tp):
+ return print(x,tp);
+ case Bag(`tp):
+ if (x instanceof MR_dataset) {
+ DataSet ds = ((MR_dataset)x).dataset();
+ List<MRData> vals = ds.take(Config.max_bag_size_print);
+ if (vals.size() == 0)
+ return "{}";
+ String s = "{ "+print(vals.get(0),tp);
+ for ( int i = 1; i < vals.size(); i++ )
+ s += ", "+print(vals.get(i),tp);
+ if (vals.size() == Config.max_bag_size_print)
+ return s+", ... }";
+ else return s+" }";
+ } else return print(x,#<bag(`tp)>);
+ case List(`tp):
+ if (x instanceof MR_dataset) {
+ DataSet ds = ((MR_dataset)x).dataset();
+ List<MRData> vals = ds.take(Config.max_bag_size_print);
+ if (vals.size() == 0)
+ return "[]";
+ String s = "[ "+print(vals.get(0),tp);
+ for ( int i = 1; i < vals.size(); i++ )
+ s += ", "+print(vals.get(i),tp);
+ if (vals.size() == Config.max_bag_size_print)
+ return s+", ... ]";
+ else return s+" ]";
+ } else return print(x,#<list(`tp)>);
+ case bag(`tp):
+ Bag b = (Bag)x;
+ Iterator<MRData> bi = b.iterator();
+ if (!bi.hasNext())
+ return "{}";
+ String s = "{ "+print(bi.next(),tp);
+ for ( long i = 1; bi.hasNext() && (Config.max_bag_size_print < 0
+ || i < Config.max_bag_size_print); i++ )
+ s += ", "+print(bi.next(),tp);
+ if (bi.hasNext())
+ return s+", ... }";
+ else return s+" }";
+ case list(`tp):
+ Bag b = (Bag)x;
+ Iterator<MRData> bi = b.iterator();
+ if (!bi.hasNext())
+ return "[]";
+ String s = "[ "+print(bi.next(),tp);
+ for ( long i = 1; bi.hasNext() && (Config.max_bag_size_print < 0
+ || i < Config.max_bag_size_print); i++ )
+ s += ", "+print(bi.next(),tp);
+ if (bi.hasNext())
+ return s+", ... ]";
+ else return s+" ]";
+ case tuple(...el):
+ Tuple t = (Tuple)x;
+ if (t.size() == 0)
+ return "()";
+ String s = "("+print(t.get((short)0),el.nth(0));
+ for ( short i = 1; i < t.size(); i++ )
+ s += ","+print(t.get(i),el.nth(i));
+ return s+")";
+ case record(...el):
+ Tuple t = (Tuple)x;
+ if (t.size() == 0)
+ return "<>";
+ String s = "< ";
+ match el.nth(0) {
+ case bind(`a,`tp):
+ s += a+": "+print(t.get((short)0),tp);
+ };
+ for ( short i = 1; i < t.size(); i++ )
+ match el.nth(i) {
+ case bind(`a,`tp):
+ s += ", "+a+": "+print(t.get(i),tp);
+ };
+ return s+" >";
+ case union(...el):
+ Union u = (Union)x;
+ match el.nth(u.tag()) {
+ case `c(tuple(...ts)):
+ return c+print(u.value(),#<tuple(...ts)>);
+ case `c(`tp):
+ return c+"("+print(u.value(),tp)+")";
+ }
+ };
+ return x.toString();
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+
+ private final static String tab ( int n ) {
+ String s = "";
+ for ( int i = 0; i < n; i++ )
+ s += " ";
+ return s;
+ }
+
+ /** print a physical plan
+ * @param e the plan
+ * @param n tab (# of spaces to put in the beginning of the line)
+ * @param pv is this a variable bound to physical plan?
+ * @return the printout
+ */
+ public final static String print_plan ( Tree e, int n, boolean pv ) {
+ match e {
+ case cMap(`f,`s):
+ return "cMap:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
+ case AggregateMap(`f,`a,`z,`s):
+ return "AggregateMap:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
+ case MapReduce(`m,`r,`s,_):
+ return "MapReduce:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
+ case MapCombineReduce(`m,`c,`r,`s,_):
+ return "MapCombineReduce:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
+ case MapAggregateReduce(`m,`r,`a,`z,`s,_):
+ return "MapAggregateReduce:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
+ case MapReduce2(`mx,`my,`r,`x,`y,_):
+ return "MapReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,_):
+ return "MapCombineReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case MapAggregateReduce2(`mx,`my,`r,`a,null,`x,`y,...):
+ return "MapReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case MapAggregateReduce2(`mx,`my,`r,`a,`z,`x,`y,...):
+ return "MapAggregateReduce2:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case MapJoin(`kx,`ky,`r,`x,`y):
+ return "MapJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case MapAggregateJoin(`kx,`ky,`r,`a,null,`x,`y):
+ return "MapJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case MapAggregateJoin(`kx,`ky,`r,`a,`z,`x,`y):
+ return "MapAggregateJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case GroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,_):
+ return "GroupByJoin:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return "CrossProduct:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case CrossAggregateProduct(`mx,`my,`r,`a,null,`x,`y):
+ return "CrossProduct:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case CrossAggregateProduct(`mx,`my,`r,`a,`z,`x,`y):
+ return "CrossAggregateProduct:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case Aggregate(`a,`z,`s):
+ return "Aggregate:\n"+tab(n+3)+"input: "+print_plan(s,n+10,true);
+ case BinarySource(`k,`file,_):
+ return "Source (binary): "+file;
+ case BinarySource(`file,_):
+ return "Source (binary): "+file;
+ case ParsedSource(`m,`parser,`file,...args):
+ if (m instanceof LongLeaf)
+ return "Source ("+parser+"): "+file;
+ else fail
+ case ParsedSource(`parser,`file,...args):
+ return "Source ("+parser+"): "+file;
+ case Generator(...):
+ return "Generator";
+ case Merge(`x,`y):
+ return "Merge:\n"+tab(n+3)+"left: "+print_plan(x,n+9,true)+"\n"
+ +tab(n+3)+"right: "+print_plan(y,n+10,true);
+ case BSP(_,_,_,_,...ds):
+ String ret = "BSP:\n";
+ for ( Tree d: ds )
+ ret += tab(n+3)+"input: "+print_plan(d,n+10,true);
+ return ret;
+ case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),...):
+ String ret = "Loop ("+vs+"):\n"+tab(n+3)+"init: "
+ +vs.head()+" = "+print_plan(ss.head(),n+vs.head().toString().length()+12,true)+"\n";
+ ss = ss.tail();
+ for ( Trees s = vs.tail(); !s.is_empty() && !ss.is_empty(); s = s.tail(), ss = ss.tail() )
+ ret += tab(n+9)+s.head()+" = "+print_plan(ss.head(),n+s.head().toString().length()+12,true)+"\n";
+ ret += tab(n+3)+"step: "
+ +vs.head()+" = "+print_plan(bs.head(),n+vs.head().toString().length()+12,true)+"\n";
+ bs = bs.tail();
+ for ( Trees s = vs.tail(); !s.is_empty() && !bs.is_empty(); s = s.tail(), bs = bs.tail() )
+ ret += tab(n+9)+s.head()+" = "+print_plan(bs.head(),n+s.head().toString().length()+12,true)+"\n";
+ return ret;
+ case `f(lambda(`v,`b),`s,...):
+ if (! #[Repeat,repeat,Closure,closure].member(#<`f>))
+ fail;
+ return f+" ("+v+"):\n"+tab(n+3)+"init: "+print_plan(s,n+9,true)+"\n"
+ +tab(n+3)+"step: "+print_plan(b,n+9,true);
+ case Let(`v,`u,`body):
+ return "let "+v+" = "+print_plan(u,n+10+v.toString().length(),pv)+"\n"
+ +tab(n)+print_plan(body,n,pv);
+ case If(_,`x1,If(_,`x2,If(_,`x3,`x4))):
+ return "Choice 1: "+print_plan(x1,n+10,pv)+"\n"
+ +tab(n)+"Choice 2: "+print_plan(x2,n+10,pv)+"\n"
+ +tab(n)+"Choice 3: "+print_plan(x3,n+10,pv)+"\n"
+ +tab(n)+"Choice 4: "+print_plan(x4,n+10,pv);
+ case If(_,`x1,If(_,`x2,`x3)):
+ return "Choice 1: "+print_plan(x1,n+10,pv)+"\n"
+ +tab(n)+"Choice 2: "+print_plan(x2,n+10,pv)+"\n"
+ +tab(n)+"Choice 3: "+print_plan(x3,n+10,pv);
+ case If(`c,`x,`y):
+ return "Choice 1: "+print_plan(x,n+10,pv)+"\n"
+ +tab(n)+"Choice 2: "+print_plan(y,n+10,pv);
+ case `f(...as):
+ String s = "";
+ for (Tree a: as) {
+ String ps = print_plan(a,n,pv);
+ if (!ps.equals("") && !a.is_variable())
+ s += ps+(ps.endsWith("\n")?"":"\n");
+ };
+ return s;
+ };
+ if (pv && e.is_variable())
+ return e.toString();
+ return "";
+ }
+
+ /** given an MRData value, construct an expression that builds this data
+ * @param x the MRData
+ * @param type the type of x
+ * @return an expression that constructs x
+ */
+ public final static Tree reify ( final MRData x, Tree type ) {
+ if (x instanceof MR_variable)
+ return new VariableLeaf("t_"+((MR_variable)x).var_num);
+ type = TypeInference.expand(type);
+ match type {
+ case `T(`tp):
+ if (!Translator.is_collection(T))
+ fail;
+ Bag b = (Bag)x;
+ Trees as = #[];
+ for ( MRData e: b)
+ as = as.append(reify(e,tp));
+ return #<`T(...as)>;
+ case tuple(...el):
+ Tuple t = (Tuple)x;
+ Trees as = #[];
+ for ( short i = 0; i < t.size(); i++ )
+ as = as.append(reify(t.get(i),el.nth(i)));
+ return #<tuple(...as)>;
+ case record(...el):
+ Tuple t = (Tuple)x;
+ Trees as = #[];
+ for ( short i = 0; i < t.size(); i++ )
+ match el.nth(i) {
+ case bind(`a,`tp):
+ as = as.append(#<bind(`a,`(reify(t.get(i),tp)))>);
+ };
+ return #<record(...as)>;
+ case union(...el):
+ Union u = (Union)x;
+ match el.nth(u.tag()) {
+ case `c(tuple(...ts)):
+ return #<call(`c,`(reify(u.value(),#<tuple(...ts)>)))>;
+ case `c(`tp):
+ return #<call(`c,`(reify(u.value(),tp)))>;
+ };
+ case string:
+ String[] s = ((MR_string)x).get().split("\\x7B\\x7B");
+ if (s.length == 1)
+ return new StringLeaf(s[0]);
+ Trees as = s[0].length() == 0 ? #[] : #[].append(new StringLeaf(s[0]));
+ for ( int i = 1; i < s.length; i++ ) {
+ String[] d = s[i].split("\\x7D\\x7D",2);
+ if (d.length < 2)
+ throw new Error("");
+ as = as.append(new VariableLeaf("t_"+Integer.parseInt(d[0])));
+ if (d[1].length() > 0)
+ as = as.append(new StringLeaf(d[1]));
+ };
+ Tree res = as.reverse().head();
+ for ( Tree a: as.reverse().tail() )
+ res = #<call(plus,`a,`res)>;
+ return res;
+ case short: return #<typed(`(((MR_short)x).get()),`type)>;
+ case int: return #<typed(`(((MR_int)x).get()),`type)>;
+ case long: return #<typed(`((int)((MR_long)x).get()),`type)>;
+ case float: return #<typed(`(((MR_float)x).get()),`type)>;
+ case double: return #<typed(`((float)(((MR_double)x).get())),`type)>;
+ };
+ throw new Error("wrong type: "+type);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/QueryPlan.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/QueryPlan.gen b/core/src/main/java/org/apache/mrql/QueryPlan.gen
new file mode 100644
index 0000000..d9b907f
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/QueryPlan.gen
@@ -0,0 +1,999 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.*;
+
+
+/** Optimize a query plan by constructing a query graph and use a greedy
+ * graph reduction algorithm to construct the query plan.
+ * More details at: http://lambda.uta.edu/dood97.pdf
+ */
+final public class QueryPlan {
+ private static Hashtable<String,Tree> repeat_plans;
+
+private final static class SingleQueryPlan {
+ static Tree[][] predicate; // the join predicate between two nodes
+ static double[][] selectivity; // the selectivity of the join predicate
+ static Tree[] plan; // the node plan
+ static String[] var; // variable name
+ static BitSet[] variables; // node variables
+ static BitSet[] depends; // node dependencies
+ static int[] depth; // min nesting level of the node
+ static Tree[] pattern; // the pattern tree associated with the node
+ static double[] size; // node cardinality
+ static Tree[] filter; // the filter of a leaf dataset
+ static Tree header; // the header of the root operator
+ static Trees query_variables;// all query variables
+ static SymbolTable header_binds; // the query header variables
+ static HashMap<String,Integer> depths; // variable depths
+ static boolean no_grouping; // true if we don't nest the operation results
+
+ /** generate a fresh variable */
+ static Tree new_var () { return Translator.new_var(); }
+
+ /** true if the query domain is a collection retrieved from a data source */
+ static boolean persistent_domain ( Tree e, Trees vars ) {
+ if (contains_variables(e,vars)) // dependent to a persistent collection
+ return true;
+ match TypeInference.type_inference2(e) {
+ case `T(_):
+ if (Translator.is_persistent_collection(T)) // persistent collection
+ return true;
+ };
+ return false;
+ }
+
+ /** true if the query domain is a collection retrieved from a data source */
+ static boolean persistent_domain ( Tree e ) {
+ return persistent_domain(e,query_variables);
+ }
+
+ /** the query bindings at any nesting level */
+ static Trees all_binds ( Tree e, Trees vars ) {
+ match e {
+ case select(`u,from(...bl),where(`p)):
+ Trees nl = #[];
+ Trees vs = vars;
+ for ( Tree b: bl )
+ match b {
+ case bind(`v,`d):
+ nl = nl.append(all_binds(b,vs));
+ if (persistent_domain(d,vs)) {
+ vs = vs.append(v);
+ nl = nl.append(b);
+ }
+ };
+ return nl.append(all_binds(p,vs)).append(all_binds(u,vs));
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(all_binds(a,vars));
+ return bl;
+ };
+ return #[];
+ }
+
+ static int var_index ( String name ) {
+ for ( int i = 0; i < var.length; i++ )
+ if (name.equals(var[i]))
+ return i;
+ return -1;
+ }
+
+ static void find_dependencies ( int i, Tree e ) {
+ match e {
+ case `f(...al):
+ for (Tree a: al)
+ find_dependencies(i,a);
+ case `v:
+ if (!v.is_variable())
+ fail;
+ String nm = ((VariableLeaf)v).value();
+ int j = var_index(nm);
+ if (j >= 0)
+ depends[i].set(j);
+ }
+ }
+
+ static int find_var ( Tree e ) {
+ match e {
+ case `f(...al):
+ int i = -1;
+ for (Tree a: al) {
+ int j = find_var(a);
+ if (j == -2)
+ return j;
+ else if (j >= 0)
+ if (i >= 0 && i != j)
+ i = -2;
+ else i = j;
+ };
+ return i;
+ case `v:
+ if (!v.is_variable())
+ fail;
+ String nm = ((VariableLeaf)v).value();
+ return var_index(nm);
+ };
+ return -1;
+ }
+
+ static boolean has_select ( Tree e ) {
+ match e {
+ case select(...):
+ return true;
+ case `f(...al):
+ for (Tree a: al)
+ if (has_select(a))
+ return true;
+ };
+ return false;
+ }
+
+ static boolean contains_variables ( Tree e, Trees vars ) {
+ match e {
+ case `f(...al):
+ for (Tree a: al)
+ if (contains_variables(a,vars))
+ return true;
+ case _:
+ if (vars.member(e))
+ return true;
+ };
+ return false;
+ }
+
+ static Trees union ( Trees xs, Trees ys ) {
+ Trees s = xs;
+ for (Tree y: ys)
+ if (!s.member(y))
+ s = s.append(y);
+ return s;
+ }
+
+ static Trees difference ( Trees xs, Trees ys ) {
+ Trees s = #[];
+ for (Tree x: xs)
+ if (!ys.member(x))
+ s = s.append(x);
+ return s;
+ }
+
+ static Tree prime_expr ( Trees vars, Tree e ) {
+ match e {
+ case lambda(`p,`b):
+ return #<lambda(`p,`(prime_expr(difference(vars,pattern_variables(p)),b)))>;
+ case `f(...al):
+ Trees s = #[];
+ for (Tree a: al)
+ s = s.append(prime_expr(vars,a));
+ return #<`f(...s)>;
+ case `v:
+ if (v.is_variable())
+ if (vars.member(v))
+ return new VariableLeaf(((VariableLeaf)v).value()+"'");
+ };
+ return e;
+ }
+
+ static Trees pattern_variables ( Tree pat ) {
+ match pat {
+ case `f(...al):
+ Trees s = #[];
+ for (Tree a: al)
+ s = s.append(pattern_variables(a));
+ return s;
+ case `v:
+ if (v.is_variable())
+ return #[`v];
+ };
+ return #[];
+ }
+
+ static Tree prime ( Tree pat, Tree e ) {
+ return prime_expr(pattern_variables(pat),e);
+ }
+
+ static Tree subst_expr ( Tree var, Tree value, Tree e ) {
+ match e {
+ case lambda(`p,`b):
+ if (pattern_variables(p).member(var))
+ return e;
+ else return #<lambda(`p,`(subst_expr(var,value,b)))>;
+ case `f(...al):
+ Trees s = #[];
+ for (Tree a: al)
+ s = s.append(subst_expr(var,value,a));
+ return #<`f(...s)>;
+ case `v:
+ if (v.is_variable())
+ if (v.equals(var))
+ return value;
+ };
+ return e;
+ }
+
+ static Tree and ( Tree x, Tree y ) {
+ if (x.equals(#<true>))
+ return y;
+ else if (y.equals(#<true>))
+ return x;
+ else return #<call(and,`x,`y)>;
+ }
+
+ static Tree find_predicates ( Tree e, Trees exclude_variables ) {
+ match e {
+ case call(and,`x,`y):
+ return and(find_predicates(x,exclude_variables),
+ find_predicates(y,exclude_variables));
+ case call(eq,`x,`y):
+ if (contains_variables(x,exclude_variables)
+ || contains_variables(y,exclude_variables))
+ fail;
+ int i = find_var(x);
+ int j = find_var(y);
+ if (i >= 0 && j >= 0 && i != j) {
+ predicate[i][j] = predicate[j][i]
+ = (predicate[i][j].equals(#<true>)) ? e : and(e,predicate[i][j]);
+ selectivity[i][j] = selectivity[j][i] = 0.01;
+ return #<true>;
+ } else if (i >= 0 && j == -1) {
+ filter[i] = (filter[i].equals(#<true>)) ? e : and(e,filter[i]);
+ plan[i] = #<cmap(lambda(`(var[i]),if(`e,bag(`(var[i])),bag())),`(plan[i]))>;
+ return #<true>;
+ } else if (j >= 0 && i == -1) {
+ filter[j] = (filter[j].equals(#<true>)) ? e : and(e,filter[j]);
+ plan[j] = #<cmap(lambda(`(var[j]),if(`e,bag(`(var[j])),bag())),`(plan[j]))>;
+ return #<true>;
+ }
+ case call(`f,`x,`y):
+ if (! #[ne,gt,geq,lt,leq].member(f))
+ fail;
+ if (has_select(x) || has_select(y))
+ fail;
+ if (contains_variables(x,exclude_variables)
+ || contains_variables(y,exclude_variables))
+ fail;
+ int i = find_var(x);
+ int j = find_var(y);
+ if (i >= 0 && j < 0) {
+ filter[i] = (filter[i].equals(#<true>)) ? e : and(e,filter[i]);
+ plan[i] = #<cmap(lambda(`(var[i]),if(`e,bag(`(var[i])),bag())),`(plan[i]))>;
+ return #<true>;
+ } else if (i < 0 && j >= 0) {
+ filter[j] = (filter[j].equals(#<true>)) ? e : and(e,filter[j]);
+ plan[j] = #<cmap(lambda(`(var[j]),if(`e,bag(`(var[j])),bag())),`(plan[j]))>;
+ return #<true>;
+ }
+ };
+ return e;
+ }
+
+ static String tuple_var ( Tree x ) {
+ match x {
+ case cmap(_,`z): return tuple_var(z);
+ };
+ String s = x.toString();
+ if (s.endsWith("'"))
+ return s.substring(0,s.length()-1);
+ return s;
+ }
+
+ /** reorder the tuple components in s based on the expected pattern variables in vars */
+ static Tree tuple ( Trees s, Trees vars ) {
+ if (s.length() != vars.length())
+ throw new Error("Wrong pattern: "+s+" "+vars);
+ if (s.length() == 1)
+ return s.head();
+ Tree[] v = new Tree[s.length()];
+ for (Tree x: s) {
+ int i = 0;
+ for (Tree y: vars) {
+ if (tuple_var(x).equals(y.toString()))
+ v[i] = x;
+ i++;
+ };
+ };
+ Trees rs = #[];
+ for ( int i = v.length-1; i >= 0 ; i-- )
+ if (v[i] == null)
+ throw new Error("Wrong pattern: "+s+" "+vars);
+ else rs = rs.cons(v[i]);
+ return #<tuple(...rs)>;
+ }
+
+ static class Header {
+ public Tree header;
+ public Trees pattern;
+ Header ( Tree h, Trees p ) { header = h; pattern = p; }
+ public String toString () { return header+" "+pattern; }
+ }
+
+ static Header build_graph ( Tree e, int level ) {
+ match e {
+ case select(`u,from(...bl),where(`p)):
+ Trees nl = #[];
+ Tree nv = new_var();
+ Trees rest = #[];
+ Trees exclude_variables = #[];
+ depths.put(nv.toString(),new Integer(level));
+ for (Tree b: bl)
+ match b {
+ case bind(`v,`d):
+ if (!persistent_domain(d)) {
+ exclude_variables = exclude_variables.append(v);
+ rest = rest.append(b);
+ continue;
+ };
+ String name = ((VariableLeaf)v).value();
+ int i = var_index(name);
+ Header nd = build_graph(d,level+1);
+ plan[i] = nd.header;
+ depth[i] = level;
+ query_variables = query_variables.append(v);
+ depths.put(name,new Integer(level+1));
+ pattern[i] = #<`(nv.toString())(`v)>;
+ find_dependencies(i,d);
+ nl = nl.append(v);
+ };
+ if (nl.equals(#[]))
+ return new Header(e,#[]);
+ query_variables = query_variables.append(nv);
+ Header nu = build_graph(u,level+1);
+ Tree npred = find_predicates(p,exclude_variables);
+ Header np = build_graph(npred,level+1);
+ for (Tree b: nu.pattern)
+ match b {
+ case bind(`v,_): nl = nl.append(v);
+ };
+ for (Tree b: np.pattern)
+ match b {
+ case bind(`v,_): nl = nl.append(v);
+ };
+ Tree t = tuple(nl,nl);
+ header_binds.insert(nv.toString(),t);
+ return new Header(#<select(`(nu.header),from(bind(`t,`nv),...rest),where(`(np.header)))>,
+ #[bind(`nv,`t)]);
+ case `f(...al):
+ Trees bl = #[];
+ Trees nl = #[];
+ for (Tree a: al) {
+ Header n = build_graph(a,level);
+ bl = bl.append(n.header);
+ nl = nl.append(n.pattern);
+ };
+ return new Header(#<`f(...bl)>,nl);
+ };
+ return new Header(e,#[]);
+ }
+
+ static void dump ( int n ) {
+ System.out.println("Query graph nodes:");
+ for ( int i = 0; i < n; i++ ) {
+ System.out.print(""+i+") "+variables(i)+" depth="+depth[i]+" pattern="+pattern[i]
+ +" plan="+plan[i]+" size="+size[i]+" depends=(");
+ for ( int j = 0; j < n; j++ )
+ if (depends[i].get(j))
+ System.out.print(""+j+" ");
+ System.out.println(") "+filter[i]);
+ };
+ System.out.println("Query graph edges (predicates):");
+ for ( int i = 0; i < n; i++ )
+ for ( int j = 0; j < i; j++ )
+ if (!predicate[i][j].equals(#<true>))
+ System.out.println(""+i+" "+j+") "+predicate[i][j]);
+ System.out.println("----------------------");
+ }
+
+ static Trees variables ( BitSet bs ) {
+ Trees bl = #[];
+ for ( int j = bs.nextSetBit(0); j >= 0; j = bs.nextSetBit(j+1)) {
+ bl = bl.append(#<`(var[j])>);
+ };
+ return bl;
+ }
+
+ static Trees variables ( int i ) {
+ return variables(variables[i]);
+ }
+
+ static Tree make_key ( Tree pred, BitSet vars ) {
+ match pred {
+ case call(and,`x,`y):
+ return #<tuple(`(make_key(x,vars)),`(make_key(y,vars)))>;
+ case call(eq,`x,`y):
+ int i = find_var(x);
+ if (i >= 0 && vars.get(i))
+ return x;
+ else return y;
+ };
+ return pred;
+ }
+
+ static boolean eq_patterns ( Trees xs, Tree y ) {
+ Trees ys = (y.is_node()) ? ((Node)y).children() : #[`y];
+ if (xs.length() != ys.length())
+ return false;
+ for (Tree x: xs)
+ if (!ys.member(x))
+ return false;
+ return true;
+ }
+
+ static String pattern_head ( Tree x ) {
+ String s = "";
+ match x {
+ case `g(...): return g;
+ };
+ return x.toString();
+ }
+
+ static Tree pattern_head ( Tree x, boolean prime ) {
+ String s = pattern_head(x);
+ return #<`(prime ? s+"'" : s)>;
+ }
+
+ static Trees pattern_children ( Tree x ) {
+ match x {
+ case _(...r): return r;
+ };
+ throw new Error("pattern is not a set: "+x);
+ }
+
+ static Tree pattern ( Tree p, boolean prime ) {
+ Trees s = #[];
+ match p {
+ case `f(...r):
+ for (Tree x: r)
+ s = s.append(pattern_head(x,prime));
+ case _: s = s.append(pattern_head(p,prime));
+ };
+ return (s.length() == 1) ? s.head() : #<tuple(...s)>;
+ }
+
+ static Trees pattern_children_variables ( Tree p ) {
+ Trees s = #[];
+ match p {
+ case `f(...r):
+ for (Tree x: r)
+ s = s.append(pattern_head(x,false));
+ case _: s = s.append(pattern_head(p,false));
+ };
+ return s;
+ }
+
+ static boolean contains ( Tree pattern, String var ) {
+ match pattern {
+ case `f(...s):
+ if (f.equals(var))
+ return true;
+ for (Tree x: s)
+ if (contains(x,var))
+ return true;
+ };
+ return pattern.equals(#<`var>);
+ }
+
+ static Trees merge_patterns ( Tree x, Trees r ) {
+ Trees s = #[];
+ for (Tree y: r)
+ if (pattern_overlap(x,y))
+ s = s.append(merge_patterns(x,y));
+ else s = s.append(y);
+ if (!pattern_overlap(x,r))
+ s = s.append(x);
+ return s;
+ }
+
+ static Trees merge_patterns ( Trees r1, Trees r2 ) {
+ Trees s = #[];
+ for (Tree x: r1)
+ if (pattern_overlap(x,r2))
+ s = s.append(merge_patterns(x,r2));
+ else s = s.append(x);
+ for (Tree y: r2)
+ if (!pattern_overlap(y,r1))
+ s = s.append(y);
+ return s;
+ }
+
+ static Tree merge_patterns ( Tree p1, Tree p2 ) {
+ match p1 {
+ case `f1(...r1):
+ match p2 {
+ case `f2(...r2):
+ if (no_grouping || depth(f1) == depth(f2))
+ return #<`f1(...(merge_patterns(r1,r2)))>;
+ if (depth(f1) < depth(f2))
+ return #<`f1(...(merge_patterns(p2,r1)))>;
+ if (depth(f1) > depth(f2))
+ return #<`f2(...(merge_patterns(p1,r2)))>;
+ case _: return #<`f1(...(merge_patterns(p2,r1)))>;
+ };
+ case_ :
+ match p2 {
+ case `f2(...r2):
+ return #<`f2(...(merge_patterns(p1,r2)))>;
+ }
+ };
+ throw new Error("Cannot merge the pattern "+p1+" with "+p2);
+ }
+
+ static boolean pattern_overlap ( Tree x, Trees r ) {
+ for (Tree y: r)
+ if (pattern_overlap(x,y))
+ return true;
+ return false;
+ }
+
+ static boolean pattern_overlap ( Tree x, Tree y ) {
+ match x {
+ case `f1(...r1):
+ match y {
+ case `f2(...r2):
+ if (f1.equals(f2)
+ || contains(header_binds.lookup(f1),f2)
+ || contains(header_binds.lookup(f2),f1))
+ return true;
+ case _: return contains(header_binds.lookup(f1),y.toString());
+ };
+ };
+ return x.equals(y);
+ }
+
+ static Trees join_body ( Tree x, Trees r, Tree pred ) {
+ Trees s = #[];
+ if (!pattern_overlap(x,r))
+ s = s.append((pred.equals(#<true>))
+ ? pattern_head(x,false)
+ : #<cmap(lambda(`(pattern(x,false)),if(`pred,bag(`(pattern(x,false))),bag())),
+ `(pattern_head(x,false)))>);
+ for (Tree y: r)
+ if (pattern_overlap(x,y))
+ s = s.append(join_body(x,y,#<true>));
+ else s = s.append(pattern_head(y,true));
+ return s;
+ }
+
+ static Trees join_body ( Trees r, Tree y, Tree pred ) {
+ Trees s = #[];
+ for (Tree x: r)
+ if (pattern_overlap(x,y))
+ s = s.append(join_body(x,y,#<true>));
+ else s = s.append(pattern_head(x,false));
+ if (!pattern_overlap(y,r))
+ s = s.append((pred.equals(#<true>))
+ ? pattern_head(y,true)
+ : #<cmap(lambda(`(pattern(y,true)),if(`pred,bag(`(pattern(y,true))),bag())),
+ `(pattern_head(y,true)))>);
+ return s;
+ }
+
+ static Trees join_body ( Trees r1, Trees r2 ) {
+ Trees s = #[];
+ for (Tree x: r1)
+ if (pattern_overlap(x,r2))
+ s = s.append(join_body(x,r2,#<true>));
+ else s = s.append(pattern_head(x,false));
+ for (Tree y: r2)
+ if (!pattern_overlap(y,r1))
+ s = s.append(pattern_head(y,true));
+ return s;
+ }
+
+ static int depth ( String n ) {
+ return depths.get(n).intValue();
+ }
+
+ static Tree join_body ( Tree p1, Tree p2, Tree pred ) {
+ Tree pat1 = pattern(p1,false);
+ Tree pat2 = pattern(p2,true);
+ Trees vars = pattern_children_variables(merge_patterns(p1,p2));
+ match p1 {
+ case `f1(...r1):
+ match p2 {
+ case `f2(...r2):
+ if (no_grouping || depth(f1) == depth(f2)) {
+ Tree t = tuple(join_body(r1,r2),vars);
+ Tree body = (pred.equals(#<true>)) ? #<bag(`t)> : #<if(`pred,bag(`t),bag())>;
+ return #<cmap(lambda(`pat1,cmap(lambda(`pat2,`body),`(f2+"'"))),
+ `f1)>;
+ } else if (depth(f1) < depth(f2)) {
+ Tree t = tuple(join_body(r1,p2,pred),vars);
+ return #<cmap(lambda(`pat1,bag(`t)),`f1)>;
+ } else if (depth(f1) > depth(f2)) {
+ Tree t = tuple(join_body(p1,r2,pred),vars);
+ return #<cmap(lambda(`pat2,bag(`t)),`(f2+"'"))>; // 3/12/11: changed from `f2
+ }
+ }
+ };
+ throw new Error("wrong join: "+p1+" "+p2);
+ }
+
+ static Tree make_join ( int i, int j ) {
+ Tree pi = pattern(pattern[i],false);
+ Tree pj = pattern(pattern[j],false);
+ Tree keyi = make_key(predicate[i][j],variables[i]);
+ Tree keyj = make_key(predicate[i][j],variables[j]);
+ Tree left = pattern_head(pattern[i],false);
+ Tree right = pattern_head(pattern[j],true);
+ Tree body = join_body(pattern[i],pattern[j],#<true>);
+ if (Config.trace)
+ System.out.print("join "+pattern[i]+" with "+pattern[j]);
+ pattern[i] = merge_patterns(pattern[i],pattern[j]);
+ if (Config.trace)
+ System.out.println(" to get "+pattern[i]+" with body "+body);
+ return #<join(lambda(`pi,`keyi),
+ lambda(`pj,`keyj),
+ lambda(tuple(`left,`right),`body),
+ `(plan[i]),
+ `(plan[j]))>;
+ }
+
+ private static Tree top_pattern_variables ( Tree pat ) {
+ match pat {
+ case _(...ts):
+ Trees ps = #[];
+ for ( Tree t: ts )
+ match t {
+ case `f(...): ps = ps.append(#<`f>);
+ case _: ps = ps.append(t);
+ };
+ if (ps.length() > 1)
+ return #<tuple(...ps)>;
+ else return ps.head();
+ };
+ return pat;
+ }
+
+ static Tree make_unnest ( int i, int j ) {
+ Tree body = null;
+ if (Config.trace)
+ System.out.print("unnest "+pattern[i]+" -> "+pattern[j]);
+ if (!no_grouping && depth[i] < depth[j]) {
+ // Changed 6/13/13: must rearrange binding variables in nested queries based on join order
+ //body = subst_expr(pattern_head(pattern[j],false),plan[j],plan[i]);
+ body = subst_header(pattern_head(pattern[j],false),top_pattern_variables(pattern[j]),plan[j],plan[i]);
+ // new pattern[i] is the old pattern[i]
+ } else {
+ body = join_body(pattern[j],pattern[i],predicate[i][j]);
+ body = prime(pattern[i],body);
+ body = subst_expr(pattern_head(pattern[j],false),plan[j],
+ subst_expr(pattern_head(pattern[i],true),plan[i],body));
+ pattern[i] = merge_patterns(pattern[j],pattern[i]);
+ };
+ if (Config.trace)
+ System.out.println(" to get "+pattern[i]+" with body "+body);
+ return body;
+ }
+
+ static Tree make_map_join ( int i, int j ) {
+ Tree body = join_body(pattern[i],pattern[j],predicate[i][j]);
+ Tree left = pattern_head(pattern[i],false);
+ Tree right = pattern_head(pattern[j],true);
+ match body {
+ case cmap(lambda(`x,cmap(lambda(`y,`b),`xx)),`yy):
+ if (!xx.equals(right) || !yy.equals(left))
+ fail;
+ Tree nb = Meta.subst_expr(x,left,Meta.subst_expr(y,right,b));
+ body = #<crossProduct(lambda(x,bag(x)),
+ lambda(x,bag(x)),
+ lambda(tuple(`left,`right),`nb),
+ `(plan[i]),
+ `(plan[j]))>;
+ case cmap(lambda(`x,`b),`xx):
+ if (!xx.equals(left))
+ fail;
+ body = Meta.subst_expr(x,xx,b);
+ body = #<groupBy(crossProduct(lambda(x,bag(x)),
+ lambda(x,bag(x)),
+ lambda(tuple(`left,`right),`body),
+ `(plan[i]),
+ `(plan[j])))>;
+ case _:
+ body = prime(pattern[j],body);
+ body = subst_expr(pattern_head(pattern[j],true),plan[j],
+ subst_expr(pattern_head(pattern[i],false),plan[i],body));
+ };
+ if (Config.trace)
+ System.out.print("cross product "+pattern[i]+" with "+pattern[j]);
+ pattern[i] = merge_patterns(pattern[i],pattern[j]);
+ if (Config.trace)
+ System.out.println(" to get "+pattern[i]+" with body "+body);
+ return body;
+ }
+
+ static Tree make_plan ( int i, int j ) {
+ if (depends[i].get(j))
+ return make_unnest(i,j);
+ else if (predicate[i][j].equals(#<true>))
+ return make_map_join(i,j);
+ else return make_join(i,j);
+ }
+
+ /** node i should not have any join predicate with a node other than j */
+ static boolean no_neighbors ( int i, int j, int n ) {
+ for (int k = 0; k < n; k++)
+ if (k != i && k != j && !predicate[i][k].equals(#<true>))
+ return false;
+ return true;
+ }
+
+ static boolean eligible ( int i, int j, int n ) {
+ if (!depends[j].isEmpty()) // j must not have any dependency
+ return false;
+ else if (depends[i].isEmpty() // a join between i and j (neither i nor j have any dependency)
+ || (depends[i].nextSetBit(0) == j
+ && depends[i].nextSetBit(j+1) < 0)) { // i depends only on j
+ if (no_grouping)
+ return true;
+ else if (depth[i] == depth[j])
+ return true;
+ else if (depth[i] < depth[j])
+ return no_neighbors(j,i,n);
+ else return no_neighbors(i,j,n);
+ };
+ return false;
+ }
+
+ static Tree subst_header ( Tree var, Tree pat, Tree plan, Tree header ) {
+ match header {
+ case bind(`p,`w):
+ if (w.equals(var))
+ return #<bind(`pat,`plan)>;
+ else fail
+ case `f(...al):
+ Trees bl = #[];
+ for (Tree a: al)
+ bl = bl.append(subst_header(var,pat,plan,a));
+ return #<`f(...bl)>;
+ };
+ return header;
+ }
+
+ static Tree ordered_tuple ( Trees xs ) {
+ if (xs.length() == 1)
+ return xs.head();
+ Trees res = #[];
+ for ( Tree v: query_variables )
+ if (xs.member(v))
+ res = res.append(v);
+ return #<tuple(...res)>;
+ }
+
+ /** group-by the plan so that the flat results in xs are grouped at their proper level */
+ static Tree final_groupBy ( int level, Trees xs, Tree plan ) {
+ Trees rest = #[];
+ Trees group_by_vars = #[];
+ Trees pvars = #[];
+ for ( Tree x: xs ) {
+ String v = pattern_head(x);
+ pvars = pvars.append(#<`v>);
+ if (depth(v) == level)
+ group_by_vars = group_by_vars.cons(#<`v>);
+ else rest = rest.append(#<`v>);
+ };
+ if (!rest.is_empty()) {
+ if (pvars.is_empty())
+ return final_groupBy(level+1,rest,plan);
+ Tree nv = new_var();
+ Tree tp = (pvars.length()==1) ? pvars.head() : #<tuple(...pvars)>;
+ Tree tg = ordered_tuple(group_by_vars);
+ Tree tr = ordered_tuple(rest);
+ Tree new_plan = #<groupBy(cmap(lambda(`tp,bag(tuple(`tg,`tr))),`plan))>;
+ Tree p = final_groupBy(level+1,rest,nv);
+ return #<cmap(lambda(tuple(`tg,`nv),bag(tuple(`tg,`p))),`new_plan)>;
+ } else return plan;
+ }
+
+ static Tree final_groupBy ( Tree plan, Tree pattern ) {
+ match pattern {
+ case `f(...r):
+ return final_groupBy(1,r,plan);
+ };
+ throw new Error("Wrong pattern in final group-by: "+pattern);
+ }
+
+ /** plan cost */
+ static double cost ( int i, int j ) {
+ return size[i]*size[j]*selectivity[i][j];
+ }
+
+ public static Tree best_plan ( Tree e ) {
+ Trees binds = all_binds(e,#[]);
+ if (binds.equals(#[]))
+ return e;
+ int N = binds.length();
+ if (N==0)
+ return e;
+ predicate = new Tree[N][];
+ selectivity = new double[N][];
+ plan = new Tree[N];
+ var = new String[N];
+ variables = new BitSet[N];
+ pattern = new Tree[N];
+ depth = new int[N];
+ size = new double[N];
+ depends = new BitSet[N];
+ filter = new Tree[N];
+ depths = new HashMap<String,Integer>();
+ Trees al = binds;
+ for ( int i = 0; i < N; i++, al = al.tail() ) {
+ match al.head() {
+ case bind(`v,`d):
+ var[i] = ((VariableLeaf) v).value();
+ variables[i] = new BitSet();
+ variables[i].set(i);
+ pattern[i] = #<`v>;
+ filter[i] = #<true>;
+ };
+ predicate[i] = new Tree[N];
+ selectivity[i] = new double[N];
+ for ( int j = 0; j < N; j++ ) {
+ predicate[i][j] = #<true>;
+ selectivity[i][j] = 1.0;
+ };
+ depends[i] = new BitSet();
+ };
+ header_binds = new SymbolTable();
+ query_variables = #[];
+ Header h = build_graph(e,0);
+ for ( int i = 0; i < N; i++ ) {
+ if (depends[i].isEmpty())
+ size[i] = 1000;
+ else size[i] = 100;
+ };
+ header = h.header;
+ if (Config.trace) {
+ System.out.println("Optimizing MRQL query:\n"+e.pretty(0));
+ System.out.println("Query Header:\n"+header.pretty(0));
+ System.out.println("Query bindings:");
+ header_binds.display();
+ System.out.print("Variable/nesting: ");
+ for (String k: depths.keySet())
+ System.out.print(k+"/"+depths.get(k)+" ");
+ System.out.println();
+ dump(N);
+ };
+ no_grouping = false;
+ for ( int n = N; n > 1; n-- ) {
+ int mi = -1;
+ int mj = -1;
+ double min = Double.MAX_VALUE;
+ for ( int i = 0; i < n; i++ )
+ for ( int j = 0; j < n; j++ ) {
+ if (i != j && eligible(i,j,n)) {
+ double cost = cost(i,j);
+ if (Config.trace)
+ System.out.println("Cost "+i+" "+j+" = "+cost);
+ if (cost < min) {
+ min = cost;
+ mi = i;
+ mj = j;
+ }
+ }
+ };
+ if (mi < 0 || mj < 0) {
+ // irreducible graph;
+ // from now on, we operate without grouping and we group-by at the end
+ if (Config.trace)
+ System.out.println("Switching to flat mode (no grouping during operations)");
+ no_grouping = true;
+ n++;
+ continue;
+ };
+ if (Config.trace)
+ System.out.println("Reduce "+mi+" with "+mj+" into "+mi);
+ // merge node mi with node mj into node mi
+ plan[mi] = make_plan(mi,mj);
+ depth[mi] = Math.min(depth[mi],depth[mj]);
+ variables[mi].or(variables[mj]);
+ size[mi] = size[mi]*size[mj]*selectivity[mi][mj];
+ depends[mi].clear(mj);
+ filter[mi] = #<true>;
+ for ( int k = 0; k < n; k++ )
+ if (k != mi) {
+ selectivity[mi][k] = selectivity[k][mi] = selectivity[mi][k]*selectivity[mj][k];
+ predicate[mi][k] = predicate[k][mi] = and(predicate[mi][k],predicate[mj][k]);
+ if (depends[k].get(mj)) {
+ depends[k].clear(mj);
+ depends[k].set(mi);
+ }
+ };
+ // replace node mj with node n-1 (last node)
+ plan[mj] = plan[n-1];
+ depth[mj] = depth[n-1];
+ pattern[mj] = pattern[n-1];
+ filter[mj] = filter[n-1];
+ depends[mj] = depends[n-1];
+ variables[mj] = variables[n-1];
+ for ( int k = 0; k < n-1; k++ )
+ if (k != mj) {
+ selectivity[mj][k] = selectivity[k][mj] = selectivity[n-1][k];
+ predicate[mj][k] = predicate[k][mj] = predicate[n-1][k];
+ if (depends[k].get(n-1)) {
+ depends[k].clear(n-1);
+ depends[k].set(mj);
+ }
+ };
+ size[mj] = size[n-1];
+ if (Config.trace)
+ dump(n-1);
+ };
+ // forced group-by
+ if (no_grouping) {
+ plan[0] = final_groupBy(plan[0],pattern[0]);
+ if (h.pattern.length() == 1)
+ match h.pattern.head() {
+ case bind(`v,`p):
+ return Meta.subst_expr(v,plan[0],h.header);
+ };
+ };
+ Tree np = pattern(pattern[0],false);
+ if (h.pattern.length() == 1)
+ match h.pattern.head() {
+ case bind(`v,`p):
+ return subst_header(v,np,plan[0],h.header);
+ };
+ return e;
+ }
+}
+
+ private static Tree process_repeat_plan ( Tree e ) {
+ match e {
+ case repeat(lambda(`x,`step),`init,...r):
+ Tree ns = SingleQueryPlan.best_plan(step);
+ repeat_plans.put(x.toString(),ns);
+ return #<repeat(lambda(`x,step(`x)),`init,...r)>;
+ case closure(lambda(`x,`step),`init,...r):
+ Tree ns = SingleQueryPlan.best_plan(step);
+ repeat_plans.put(x.toString(),ns);
+ return #<closure(lambda(`x,cstep(`x)),`init,...r)>;
+ case `f(...al):
+ Trees bl = #[];
+ for (Tree a: al)
+ bl = bl.append(process_repeat_plan(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ private static Tree process_nested_plan ( Tree e ) {
+ match e {
+ case select(`u,from(...bl),where(`p)):
+ return SingleQueryPlan.best_plan(e);
+ case `f(...al):
+ Trees bl = #[];
+ for (Tree a: al)
+ bl = bl.append(process_nested_plan(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ public static Tree best_plan ( Tree e ) {
+ repeat_plans = new Hashtable<String,Tree>();
+ Tree np = process_nested_plan(process_repeat_plan(e));
+ for ( String s: repeat_plans.keySet() )
+ np = Meta.subst_expr(#<step(`s)>,repeat_plans.get(s),
+ Meta.subst_expr(#<cstep(`s)>,repeat_plans.get(s),np));
+ return np;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Simplification.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Simplification.gen b/core/src/main/java/org/apache/mrql/Simplification.gen
new file mode 100644
index 0000000..f18b17a
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Simplification.gen
@@ -0,0 +1,389 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.*;
+import java.io.*;
+
+
+/** simplify algebraic forms using heuristic rewriting rules that improve performance in most cases */
+public class Simplification extends Normalization {
+
+ /* true if x is functional dependent on y (ie, equal x's implies equal y's) */
+ private static boolean functional_dependent ( Tree x, Tree y ) {
+ if (x.equals(y))
+ return true;
+ match y {
+ case tuple(...ts):
+ for ( Tree t: ts )
+ if (functional_dependent(x,t))
+ return true;
+ case record(...rs):
+ for ( Tree r: rs )
+ match r {
+ case bind(_,`t):
+ if (functional_dependent(x,t))
+ return true;
+ }
+ };
+ return false;
+ }
+
+ private static boolean simple_accessor ( Tree v, Tree e ) {
+ match e {
+ case nth(`u,`n): return simple_accessor(v,u);
+ case project(`u,`a): return simple_accessor(v,u);
+ };
+ return e.equals(v);
+ }
+
+ private static Trees factor_out_aggr ( Tree e, Tree v ) {
+ match e {
+ case call(`g,cmap(`f,`d)):
+ if (!simple_accessor(v,d) || !free_variables(f,#[]).is_empty())
+ fail;
+ for ( Tree monoid: monoids )
+ match monoid {
+ case `aggr(...):
+ if (aggr.equals(g.toString()))
+ return #[bind(`e,`d)];
+ };
+ fail
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(factor_out_aggr(a,v));
+ return bl;
+ };
+ return #[];
+ }
+
+ private static Trees factor_out_aggregations ( Tree e, Tree v ) {
+ Trees al = factor_out_aggr(e,v);
+ Tree t = e;
+ for ( Tree a: al )
+ match a {
+ case bind(`u,`d): t = subst(u,#<0>,t);
+ };
+ Trees bl = #[];
+ for ( Tree a: al )
+ match a {
+ case bind(`u,`d):
+ if (occurences(d,t) == 0) {
+ boolean exists = false;
+ for ( Tree b: bl )
+ match b {
+ case bind(`uu,`dd):
+ if (d.equals(dd) && !u.equals(uu))
+ if (alpha_equivalent(u,uu))
+ exists = true;
+ };
+ if (true || !exists)
+ bl = bl.append(a);
+ }
+ };
+ return bl;
+ }
+
+ /** Algebraic normalization (algebra to algebra)
+ * @param e algebraic expression
+ * @return an improved algebraic expression
+ */
+ public static Tree simplify ( Tree e ) {
+ match e {
+ case cmap(`f,cmap(lambda(`x,`g),`s)):
+ return simplify(#<cmap(lambda(`x,cmap(`f,`g)),`s)>);
+ case map(`f,cmap(lambda(`x,`g),`s)):
+ return simplify(#<cmap(lambda(`x,map(`f,`g)),`s)>);
+ case cmap(`g,join(`k1,`k2,lambda(`p,`f),`X,`Y)):
+ return simplify(#<join(`k1,`k2,lambda(`p,cmap(`g,`f)),`X,`Y)>);
+ case cmap(lambda(`x,`S(`y)),`u):
+ if (is_collection(S) && x.equals(y))
+ return simplify(u);
+ else fail
+ case cmap(lambda(`x,`b),`S(`a)):
+ if (is_collection(S) && x.is_variable())
+ return simplify(subst_var(x,a,b));
+ else fail
+ case cmap(`f,`S()):
+ if (is_collection(S))
+ return #<`S()>;
+ else fail
+ case cmap(lambda(`x,`T(`b)),`S(...as)):
+ if (is_collection(S) && is_collection(T) && x.is_variable()) {
+ Trees bs = #[];
+ for ( Tree a: as )
+ bs = bs.append(simplify(subst_var(x,a,b)));
+ return #<`T(...bs)>;
+ } else fail
+ case map(lambda(`x,`b),`S(`a)):
+ if (is_collection(S) && x.is_variable())
+ return #<`S(`(simplify(subst_var(x,a,b))))>;
+ else fail
+ case map(`f,`S()):
+ if (is_collection(S))
+ return #<`S()>;
+ else fail
+ case filter(lambda(`x,`b),`m,`S(`a)):
+ if (is_collection(S) && x.is_variable())
+ return simplify(#<if(`(subst_var(x,a,b)),apply(`m,`a),`S())>);
+ else fail
+ case filter(`p,`m,`S()):
+ if (is_collection(S))
+ return #<`S()>;
+ else fail
+ case cmap(`f,if(`p,`x,`y)):
+ return simplify(#<if(`p,cmap(`f,`x),cmap(`f,`y))>);
+ // if the join reducer contains an independent aggregation push it to the input
+ // NOTE: This disables the self-join to MR transformation used in pagerank
+ case xxxjoin(`kx,`ky,
+ lambda(`v,cmap(lambda(`v1,cmap(lambda(`v2,bag(tuple(`k,`b))),
+ nth(`vy,1))),
+ nth(`vx,0))),
+ `X,`Y):
+ if (!vx.equals(v) || !vy.equals(v))
+ fail;
+ Trees l1 = factor_out_aggregations(b,v1);
+ Trees l2 = factor_out_aggregations(b,v2);
+ if (l1.is_empty() && l2.is_empty())
+ fail;
+ Tree px = v1;
+ Trees dl = #[];
+ for ( Tree a: l1 )
+ match a {
+ case bind(`u,`d):
+ Tree nv = new_var();
+ b = subst(u,nv,b);
+ if (true || !dl.member(d)) {
+ Tree vv = new_var();
+ X = #<cmap(lambda(`px,bag(tuple(`px,`u))),`X)>;
+ px = #<tuple(`px,`nv)>;
+ }
+ dl = dl.append(d);
+ };
+ Tree py = v2;
+ dl = #[];
+ for ( Tree a: l2 )
+ match a {
+ case bind(`u,`d):
+ Tree nv = new_var();
+ b = subst(u,nv,b);
+ if (true || !dl.member(d)) {
+ Tree vv = new_var();
+ Y = #<cmap(lambda(`py,bag(tuple(`py,`u))),`Y)>;
+ py = #<tuple(`py,`nv)>;
+ };
+ dl = dl.append(d);
+ };
+ Tree res = #<join(lambda(`px,apply(`kx,`v1)),
+ lambda(`py,apply(`ky,`v2)),
+ lambda(`v,cmap(lambda(`px,cmap(lambda(`py,bag(tuple(`k,`b))),
+ nth(`vy,1))),
+ nth(`vx,0))),
+ `X,`Y)>;
+ res = rename(res);
+ return simplify(res);
+ // if the reducer of a join generates pairs (k,v), where k is functional dependent
+ // on a join key, then the outer groupBy just groups the v values
+ case groupBy(join(lambda(`vx,`bx),`ky,
+ lambda(`v,cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ex,`br))),
+ nth(`v1,1))),
+ nth(`v2,0))),
+ `X,`Y)):
+ if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vx,x,bx),ex))
+ return simplify(#<join(lambda(`vx,`bx),`ky,
+ lambda(`v,groupBy(cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ex,`br))),
+ nth(`v1,1))),
+ nth(`v2,0)))),
+ `X,`Y)>);
+ fail
+ // same for the right key
+ case groupBy(join(`kx,lambda(`vy,`by),
+ lambda(`v,cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ey,`br))),
+ nth(`v1,1))),
+ nth(`v2,0))),
+ `X,`Y)):
+ if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vy,y,by),ey))
+ return simplify(#<join(`kx,lambda(`vy,`by),
+ lambda(`v,groupBy(cmap(lambda(`x,cmap(lambda(`y,bag(tuple(`ey,`br))),
+ nth(`v1,1))),
+ nth(`v2,0)))),
+ `X,`Y)>);
+ fail
+ // same for the left key, different nesting
+ case groupBy(join(lambda(`vx,`bx),`ky,
+ lambda(`v,cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ex,`br))),
+ nth(`v1,0))),
+ nth(`v2,1))),
+ `X,`Y)):
+ if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vx,x,bx),ex))
+ return simplify(#<join(lambda(`vx,`bx),`ky,
+ lambda(`v,groupBy(cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ex,`br))),
+ nth(`v1,1))),
+ nth(`v2,0)))),
+ `X,`Y)>);
+ fail
+ // same for the right key, different nesting
+ case groupBy(join(`kx,lambda(`vy,`by),
+ lambda(`v,cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ey,`br))),
+ nth(`v1,0))),
+ nth(`v2,1))),
+ `X,`Y)):
+ if (v1.equals(v) && v2.equals(v) && functional_dependent(subst(vy,y,by),ey))
+ return simplify(#<join(`kx,lambda(`vy,`by),
+ lambda(`v,groupBy(cmap(lambda(`y,cmap(lambda(`x,bag(tuple(`ey,`br))),
+ nth(`v1,0))),
+ nth(`v2,1)))),
+ `X,`Y)>);
+ fail
+ // same for the left key, right nested
+ case groupBy(join(lambda(`vx,`bx),`ky,
+ lambda(`v,cmap(lambda(`x,bag(tuple(`ex,`br))),
+ nth(`v1,0))),
+ `X,`Y)):
+ if (v1.equals(v) && functional_dependent(subst(vx,x,bx),ex))
+ return simplify(#<join(lambda(`vx,`bx),`ky,
+ lambda(`v,groupBy(cmap(lambda(`x,bag(tuple(`ex,`br))),
+ nth(`v1,0)))),
+ `X,`Y)>);
+ fail
+ // same for the right key, left nested
+ case groupBy(join(`kx,lambda(`vy,`by),
+ lambda(`v,cmap(lambda(`y,bag(tuple(`ey,`br))),
+ nth(`v2,1))),
+ `X,`Y)):
+ if (v2.equals(v) && functional_dependent(subst(vy,y,by),ey))
+ return simplify(#<join(`kx,lambda(`vy,`by),
+ lambda(`v,groupBy(cmap(lambda(`y,bag(tuple(`ey,`br))),
+ nth(`v2,1)))),
+ `X,`Y)>);
+ fail
+ // if we group-by the join key, then embed the group-by in the join reducer
+ // (redundant rule)
+ case groupBy(join(`kx,`ky,lambda(`v,cmap(lambda(`v1,cmap(lambda(`v2,bag(tuple(`k,`u))),`e1)),`e2)),
+ `X,`Y)):
+ if (((e1.equals(#<nth(`v,0)>) && e2.equals(#<nth(`v,1)>))
+ || (e2.equals(#<nth(`v,0)>) && e1.equals(#<nth(`v,1)>)))
+ && (alpha_equivalent(kx,#<lambda(`v1,`k)>)
+ || alpha_equivalent(kx,#<lambda(`v2,`k)>)))
+ return simplify(#<join(`kx,`ky,lambda(`v,groupBy(cmap(lambda(`v1,cmap(lambda(`v2,
+ bag(tuple(`k,`u))),`e1)),`e2))),
+ `X,`Y)>);
+ fail
+ case groupBy(groupBy(`x)):
+ Tree nv = new_var();
+ return simplify(#<cmap(lambda(`nv,bag(bag(`nv))),groupBy(`x))>);
+ case repeat(lambda(`v,`b),`s,...l):
+ repeat_variables = repeat_variables.cons(v);
+ return #<repeat(lambda(`v,`(simplify(b))),`(simplify(s)),...l)>;
+ case closure(lambda(`v,`b),`s,...l):
+ repeat_variables = repeat_variables.cons(v);
+ return #<closure(lambda(`v,`(simplify(b))),`(simplify(s)),...l)>;
+ case loop(lambda(tuple(...vs),`b),`s,`n):
+ repeat_variables = repeat_variables.append(vs);
+ return #<loop(lambda(tuple(...vs),`(simplify(b))),`(simplify(s)),`n)>;
+ case aggregate(`acc,`zero,`T()):
+ if (is_collection(T))
+ return zero;
+ else fail
+ case aggregate(`acc,`zero,`T(`s)):
+ if (is_collection(T))
+ return simplify(#<apply(`acc,tuple(`zero,`s))>);
+ else fail
+ case apply(lambda(`v,`b),`u):
+ if (!v.is_variable())
+ fail;
+ return simplify(subst_var(v,u,b));
+ case apply(function(tuple(...el),_,`b),`u):
+ int i = 0;
+ for ( Tree a: el )
+ match a {
+ case `bind(`v,_):
+ b = subst(v,#<nth(`u,`(i++))>,b);
+ };
+ return simplify(b);
+ case call(and,true,`u): return simplify(u);
+ case call(and,`u,true): return simplify(u);
+ case call(and,false,`u):return #<false>;
+ case call(and,`u,false): return #<false>;
+ case call(or,true,`u): return #<true>;
+ case call(or,`u,true): return #<true>;
+ case call(or,false,`u): return simplify(u);
+ case call(or,`u,false): return simplify(u);
+ case call(not,true): return #<false>;
+ case call(not,false): return #<true>;
+ case if(true,`e1,`e2): return simplify(e1);
+ case if(false,`e1,`e2): return simplify(e2);
+ case call(count,cmap(lambda(`v,`S(`x)),`u)):
+ if (is_collection(S))
+ return simplify(#<call(count,`u)>);
+ else fail
+ case call(count,`groupBy(cmap(lambda(`v,`S(tuple(`x,`y))),`u))):
+ if (is_collection(S) && !y.equals(#<0>) && #[groupBy,orderBy].member(#<`groupBy>))
+ return #<call(count,groupBy(cmap(lambda(`v,`S(tuple(`x,0))),`u)))>;
+ else fail
+ case call(count,`S(...r)):
+ if (is_collection(S))
+ return #<typed(`(r.length()),long)>;
+ else fail
+ case call(`f,`S(`x)):
+ if (!is_collection(S))
+ fail;
+ for ( Tree m: monoids )
+ match m {
+ case `aggr(`mtp,`plus,`zero,`unit):
+ if (!aggr.equals(f.toString()))
+ continue;
+ if (TypeInference.unify(mtp,TypeInference.type_inference2(x)) != null)
+ return simplify(#<apply(`unit,`x)>);
+ };
+ fail
+ case nth(tuple(...al),`n):
+ if (!n.is_long())
+ fail;
+ int i = (int)n.longValue();
+ if (i >= 0 && i < al.length())
+ return simplify(al.nth(i));
+ case project(record(...bl),`a):
+ for ( Tree b: bl )
+ match b {
+ case bind(`v,`u):
+ if (v.equals(a))
+ return simplify(u);
+ };
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(simplify(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ /** Algebraic normalization (algebra to algebra) applied multiple times
+ * @param e algebraic expression
+ * @return an improved algebraic expression
+ */
+ public static Tree simplify_all ( Tree e ) {
+ Tree ne = simplify(e);
+ if (e.equals(ne))
+ return e;
+ else return simplify_all(ne);
+ }
+}
[12/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/JoinOperation.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/JoinOperation.java b/src/main/java/MapReduce/JoinOperation.java
deleted file mode 100644
index 0f962df..0000000
--- a/src/main/java/MapReduce/JoinOperation.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.Hashtable;
-import java.util.Iterator;
-import java.util.Enumeration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-
-
-/** The MapReduce2 physical operator (a reduce-side join) */
-final public class JoinOperation extends MapReducePlan {
-
- /** Container for join input values. For reduce-side join, we concatenate
- * 1 or 2 to the join key before we mix the right with the left tuples */
- public final static class JoinKey implements Writable {
- public byte tag; // 1 or 2
- public MRData key;
-
- JoinKey () {}
- JoinKey ( byte t, MRData k ) { tag = t; key = k; }
-
- public void write ( DataOutput out ) throws IOException {
- out.writeByte(tag);
- key.write(out);
- }
-
- public void readFields ( DataInput in ) throws IOException {
- tag = in.readByte();
- key = MRContainer.read(in);
- }
-
- public String toString () {
- return "<"+tag+":"+key+">";
- }
- }
-
- public final static class MRContainerJoinPartitioner extends Partitioner<JoinKey,MRContainer> {
- final public int getPartition ( JoinKey key, MRContainer value, int numPartitions ) {
- return Math.abs(key.key.hashCode()) % numPartitions;
- }
- }
-
- /** The sorting of the joined values uses ths join key for major order and tag for minor order */
- public final static class MRContainerSortComparator implements RawComparator<JoinKey> {
- int[] container_size;
-
- public MRContainerSortComparator () {
- container_size = new int[1];
- }
-
- final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
- int cmp = MRContainer.compare(x,xs+1,xl-1,y,ys+1,yl-1,container_size);
- return (cmp == 0) ? x[xs]-y[ys] : cmp;
- }
-
- final public int compare ( JoinKey x, JoinKey y ) {
- int c = x.key.compareTo(y.key);
- return (c == 0) ? x.tag-y.tag : c;
- }
- }
-
- /** The grouping of the joined values is based on join key only */
- public final static class MRContainerGroupingComparator implements RawComparator<JoinKey> {
- int[] container_size;
-
- public MRContainerGroupingComparator() {
- container_size = new int[1];
- }
-
- final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
- return MRContainer.compare(x,xs+1,xl-1,y,ys+1,yl-1,container_size);
- }
-
- final public int compare ( JoinKey x, JoinKey y ) {
- return x.key.compareTo(y.key);
- }
- }
-
- /** The left mapper for MapReduce2 */
- private final static class MapperLeft extends Mapper<MRContainer,MRContainer,JoinKey,MRContainer> {
- private static Function mx; // the left mapper function
- private static JoinKey join_key = new JoinKey((byte)2,new MR_int(0));
- private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(2));
- private static MRContainer cvalue = new MRContainer(tvalue);
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- for ( MRData e: (Bag)mx.eval(value.data()) ) {
- Tuple p = (Tuple)e;
- join_key.key = p.first();
- tvalue.set(1,p.second());
- cvalue.set(tvalue);
- context.write(join_key,cvalue);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.mapper.left"));
- mx = functional_argument(conf,code);
- } catch (Exception e) {
- throw new Error("Cannot retrieve the left mapper plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- super.cleanup(context);
- }
- }
-
- /** The right mapper for MapReduce2 */
- private final static class MapperRight extends Mapper<MRContainer,MRContainer,JoinKey,MRContainer> {
- private static Function my; // the right mapper function
- private static JoinKey join_key = new JoinKey((byte)1,new MR_int(0));
- private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(1));
- private static MRContainer cvalue = new MRContainer(tvalue);
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- for ( MRData e: (Bag)my.eval(value.data()) ) {
- Tuple p = (Tuple)e;
- join_key.key = p.first();
- tvalue.set(1,p.second());
- cvalue.set(tvalue);
- context.write(join_key,cvalue);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.mapper.right"));
- my = functional_argument(conf,code);
- } catch (Exception e) {
- throw new Error("Cannot retrieve the right mapper plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- super.cleanup(context);
- }
- }
-
- /** The reducer for MapReduce2 */
- private static class JoinReducer extends Reducer<JoinKey,MRContainer,MRContainer,MRContainer> {
- private static String counter; // a Hadoop user-defined counter used in the repeat operation
- private static Function combine_fnc; // the combiner function
- private static Function reduce_fnc; // the reduce function
- private static Bag left = new Bag(); // a cached bag of input fragments from left input
- private static Bag right = new Bag(); // a cached bag of input fragments from right input
- private static Function acc_fnc; // aggregator
- private static MRData result; // aggregation result
- private static Hashtable<MRData,MRData> hashTable; // in-mapper combiner
- private static int index;
- private static Tuple tkey = new Tuple(2);
- private static Bag tbag = new Bag(2);
- private static boolean streamed = false;
- private static Tuple pair = new Tuple(2);
- private static MRContainer ckey = new MRContainer(new MR_int(0));
- private static MRContainer cvalue = new MRContainer(new MR_int(0));
- private static MRContainer container = new MRContainer(new MR_int(0));
-
- private void write ( MRContainer key, MRData value, Context context )
- throws IOException, InterruptedException {
- if (result != null) { // aggregation
- pair.set(0,result);
- pair.set(1,value);
- result = acc_fnc.eval(pair);
- } else if (hashTable != null) {
- MRData k = ((Tuple)value).get(0);
- MRData v = ((Tuple)value).get(1);
- MRData old = hashTable.get(k);
- if (old == null) {
- if (index++ == Config.map_cache_size)
- flush_table(context);
- hashTable.put(k,v);
- } else {
- tkey.set(0,key.data());
- tbag.clear();
- tbag.add_element(v).add_element(old);
- tkey.set(1,tbag);
- for ( MRData x: (Bag)combine_fnc.eval(tkey) )
- hashTable.put(k,x); // normally, done once
- }
- } else if (counter.equals("-")) {
- container.set(value);
- context.write(key,container);
- } else { // increment the repetition counter if the repeat condition is true
- Tuple t = (Tuple)value;
- if (((MR_bool)t.second()).get())
- context.getCounter("mrql",counter).increment(1);
- container.set(t.first());
- context.write(key,container);
- }
- }
-
- protected static void flush_table ( Context context ) throws IOException, InterruptedException {
- Enumeration<MRData> en = hashTable.keys();
- while (en.hasMoreElements()) {
- MRData key = en.nextElement();
- ckey.set(key);
- MRData value = hashTable.get(key);
- cvalue.set(value);
- if (value != null)
- context.write(ckey,new MRContainer(new Tuple(key,value)));
- };
- index = 0;
- hashTable.clear();
- }
-
- @Override
- public void reduce ( JoinKey key, Iterable<MRContainer> values, Context context )
- throws IOException, InterruptedException {
- if (!streamed) {
- left.clear();
- right.clear();
- for ( MRContainer val: values ) {
- Tuple p = (Tuple)val.data();
- if (((MR_byte)p.first()).get() == 1)
- right.add(p.second());
- else left.add(p.second());
- };
- } else { // the left input is processed lazily (as a stream-based bag)
- right.clear();
- Tuple p = null;
- final Iterator<MRContainer> i = values.iterator();
- while (i.hasNext()) {
- p = (Tuple)i.next().data();
- if (((MR_byte)p.first()).get() == 2)
- break;
- right.add(p.second());
- p = null;
- };
- final Tuple data = p;
- left = new Bag(new BagIterator () {
- boolean first_time = data != null;
- public boolean hasNext () {
- return first_time || i.hasNext();
- }
- public MRData next () {
- if (!first_time) {
- Tuple t = (Tuple)i.next().data();
- assert(((MR_byte)(t.first())).get() == 2);
- return t.second();
- };
- first_time = false;
- return data.second();
- }
- });
- };
- pair.set(0,left);
- pair.set(1,right);
- for ( MRData e: (Bag)reduce_fnc.eval(pair) ) {
- ckey.set(key.key);
- write(ckey,e,context);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.reducer"));
- reduce_fnc = functional_argument(conf,code);
- streamed = PlanGeneration.streamed_MapReduce2_reducer(code);
- if (conf.get("mrql.zero") != null) {
- code = Tree.parse(conf.get("mrql.zero"));
- result = Interpreter.evalE(code);
- code = Tree.parse(conf.get("mrql.accumulator"));
- acc_fnc = functional_argument(conf,code);
- } else result = null;
- counter = conf.get("mrql.counter");
- code = Tree.parse(conf.get("mrql.combiner"));
- hashTable = null;
- if (code != null && !code.equals(new VariableLeaf("null"))) {
- combine_fnc = functional_argument(conf,code);
- hashTable = new Hashtable<MRData,MRData>(Config.map_cache_size);
- index = 0;
- }
- } catch (Exception e) {
- throw new Error("Cannot retrieve the reducer plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (result != null) // emit the result of aggregation
- context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
- if (hashTable != null)
- flush_table(context);
- hashTable = null; // garbage-collect it
- super.cleanup(context);
- }
- }
-
- /** The MapReduce2 physical operator (a reduce-side join)
- * @param mx left mapper function
- * @param my right mapper function
- * @param combine_fnc optional in-mapper combiner function
- * @param reduce_fnc reducer function
- * @param acc_fnc optional accumulator function
- * @param zero optional the zero value for the accumulator
- * @param X left data set
- * @param Y right data set
- * @param num_reduces number of reducers
- * @param stop_counter optional counter used in repeat operation
- * @param orderp does the result need to be ordered?
- * @return a new data source that contains the result
- */
- public final static DataSet mapReduce2 ( Tree mx, // left mapper function
- Tree my, // right mapper function
- Tree combine_fnc, // optional in-mapper combiner function
- Tree reduce_fnc, // reducer function
- Tree acc_fnc, // optional accumulator function
- Tree zero, // optional the zero value for the accumulator
- DataSet X, // left data set
- DataSet Y, // right data set
- int num_reduces, // number of reducers
- String stop_counter, // optional counter used in repeat operation
- boolean orderp ) // does the result need to be ordered?
- throws Exception {
- String newpath = new_path(conf);
- conf.set("mrql.mapper.left",mx.toString());
- conf.set("mrql.mapper.right",my.toString());
- if (combine_fnc != null)
- conf.set("mrql.combiner",combine_fnc.toString());
- conf.set("mrql.reducer",reduce_fnc.toString());
- if (zero != null) {
- conf.set("mrql.accumulator",acc_fnc.toString());
- conf.set("mrql.zero",zero.toString());
- // the in-mapper combiner likes large data splits
- conf.set("mapred.min.split.size","268435456"); // 256 MBs
- } else conf.set("mrql.zero","");
- conf.set("mrql.counter",stop_counter);
- Job job = new Job(conf,newpath);
- distribute_compiled_arguments(job.getConfiguration());
- job.setMapOutputKeyClass(JoinKey.class);
- job.setJarByClass(MapReducePlan.class);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setPartitionerClass(MRContainerJoinPartitioner.class);
- job.setSortComparatorClass(MRContainerSortComparator.class);
- job.setGroupingComparatorClass(MRContainerGroupingComparator.class);
- job.setOutputFormatClass(SequenceFileOutputFormat.class);
- FileOutputFormat.setOutputPath(job,new Path(newpath));
- for (DataSource p: X.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperLeft.class);
- for (DataSource p: Y.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperRight.class);
- if (Config.trace && PlanGeneration.streamed_MapReduce2_reducer(reduce_fnc))
- System.err.println("*** Streamed MapReduce2 reducer");
- job.setReducerClass(JoinReducer.class);
- if (num_reduces > 0)
- job.setNumReduceTasks(num_reduces);
- job.waitForCompletion(true);
- long c = (stop_counter.equals("-")) ? 0
- : job.getCounters().findCounter("mrql",stop_counter).getValue();
- DataSource s = new BinaryDataSource(newpath,conf);
- s.to_be_merged = orderp;
- return new DataSet(s,c,outputRecords(job));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapJoinOperation.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapJoinOperation.java b/src/main/java/MapReduce/MapJoinOperation.java
deleted file mode 100644
index 3ac4efa..0000000
--- a/src/main/java/MapReduce/MapJoinOperation.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.net.URI;
-import java.util.Hashtable;
-import java.util.Iterator;
-import java.util.Enumeration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.filecache.DistributedCache;
-
-
-/** The fragment-replicate join (map-side join) physical operator */
-final public class MapJoinOperation extends MapReducePlan {
-
- /** the mapper of the MapJoin */
- private final static class mapJoinMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
- private static String counter; // a Hadoop user-defined counter used in the repeat operation
- private static Function reduce_fnc; // the reduce function
- private static Function probe_map_fnc;
- private static Hashtable<MRData,Bag> built_table;
- private static Function acc_fnc; // aggregator
- private static MRData result; // aggregation result
- private static Tuple pair = new Tuple(2);
- private static MRContainer container = new MRContainer(new MR_int(0));
- private static Bag empty_bag = new Bag();
- private static boolean mapJoinReduce = false;
-
- private void write ( MRContainer key, MRData value, Context context )
- throws IOException, InterruptedException {
- if (result != null) { // aggregation
- pair.set(0,result);
- pair.set(1,value);
- result = acc_fnc.eval(pair);
- } else if (counter.equals("-")) {
- container.set(value);
- context.write(key,container);
- } else { // increment the repetition counter if the repeat condition is true
- Tuple t = (Tuple)value;
- if (((MR_bool)t.second()).get())
- context.getCounter("mrql",counter).increment(1);
- container.set(t.first());
- context.write(key,container);
- }
- }
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- for ( MRData e: (Bag)probe_map_fnc.eval(value.data()) ) {
- Tuple p = (Tuple)e;
- MRData pd = built_table.get(p.first());
- if (pd == null)
- pd = empty_bag;
- pair.set(0,p.second());
- pair.set(1,pd);
- for ( MRData v: (Bag)reduce_fnc.eval(pair) )
- write(key,v,context);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- if (conf.get("mrql.mapJoinReduce") != null)
- mapJoinReduce = true;
- Tree code = Tree.parse(conf.get("mrql.inMap.reducer"));
- reduce_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.probe_mapper"));
- probe_map_fnc = functional_argument(conf,code);
- if (!mapJoinReduce && conf.get("mrql.zero") != null) {
- code = Tree.parse(conf.get("mrql.zero"));
- result = Interpreter.evalE(code);
- code = Tree.parse(conf.get("mrql.accumulator"));
- acc_fnc = functional_argument(conf,code);
- } else result = null;
- counter = conf.get("mrql.counter");
- built_table = new Hashtable<MRData,Bag>(Config.map_cache_size);
- Bag res = new Bag();
- URI[] uris = DistributedCache.getCacheFiles(conf);
- Path[] local_paths = DistributedCache.getLocalCacheFiles(conf);
- final FileSystem fs = FileSystem.getLocal(conf);
- final Configuration fconf = conf;
- for ( int i = 0; i < local_paths.length; i++ ) {
- // hadoop 0.20.2 distributed cache doesn't work in stand-alone
- final Path path = (conf.get("mapred.job.tracker").equals("local"))
- ? new Path(uris[i].toString())
- : local_paths[i];
- if (path.getName().endsWith(".jar"))
- continue;
- res = res.union(new Bag(new BagIterator () {
- final SequenceFile.Reader reader = new SequenceFile.Reader(fs,path,fconf);
- final MRContainer key = new MRContainer(new MR_int(0));
- final MRContainer value = new MRContainer(new MR_int(0));
- public boolean hasNext () {
- try {
- boolean done = reader.next(key,value);
- if (!done)
- reader.close();
- return done;
- } catch (IOException e) {
- throw new Error("Cannot collect values from distributed cache");
- }
- }
- public MRData next () {
- return value.data();
- }
- }));
- };
- for ( MRData e: res ) {
- Tuple p = (Tuple)e;
- Bag entries = built_table.get(p.first());
- built_table.put(p.first(),
- (entries == null)
- ? (new Bag(p.second()))
- : entries.add_element(p.second()));
- }
- } catch (Exception e) {
- throw new Error("Cannot setup the mapJoin: "+e);
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (result != null) // emit the result of aggregation
- context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
- built_table = null; // garbage-collect it
- super.cleanup(context);
- }
- }
-
- /** The fragment-replicate join (map-side join) physical operator
- * @param probe_map_fnc left mapper function
- * @param built_map_fnc right mapper function
- * @param reduce_fnc reducer function
- * @param acc_fnc optional accumulator function
- * @param zero optional the zero value for the accumulator
- * @param probe_dataset the map source
- * @param built_dataset stored in distributed cache
- * @param stop_counter optional counter used in repeat operation
- * @return a new data source that contains the result
- */
- public final static DataSet mapJoin ( Tree probe_map_fnc, // left mapper function
- Tree built_map_fnc, // right mapper function
- Tree reduce_fnc, // reducer function
- Tree acc_fnc, // optional accumulator function
- Tree zero, // optional the zero value for the accumulator
- DataSet probe_dataset, // the map source
- DataSet built_dataset, // stored in distributed cache
- String stop_counter ) // optional counter used in repeat operation
- throws Exception {
- DataSet ds = MapOperation.cMap(built_map_fnc,null,null,built_dataset,"-");
- String newpath = new_path(conf);
- conf.set("mrql.inMap.reducer",reduce_fnc.toString());
- conf.set("mrql.probe_mapper",probe_map_fnc.toString());
- conf.set("mrql.counter",stop_counter);
- if (zero != null) {
- conf.set("mrql.accumulator",acc_fnc.toString());
- conf.set("mrql.zero",zero.toString());
- conf.set("mapred.min.split.size","268435456");
- } else conf.set("mrql.zero","");
- Job job = new Job(conf,newpath);
- distribute_compiled_arguments(job.getConfiguration());
- job.setJarByClass(MapReducePlan.class);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setOutputFormatClass(SequenceFileOutputFormat.class);
- PathFilter pf = new PathFilter () { public boolean accept ( Path path ) {
- return !path.getName().startsWith("_");
- } };
- for (DataSource p: ds.source) { // distribute the built dataset
- Path path = new Path(p.path);
- for ( FileStatus s: path.getFileSystem(conf).listStatus(path,pf) )
- DistributedCache.addCacheFile(s.getPath().toUri(),job.getConfiguration());
- };
- for (DataSource p: probe_dataset.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,mapJoinMapper.class);
- FileOutputFormat.setOutputPath(job,new Path(newpath));
- job.setNumReduceTasks(0);
- job.waitForCompletion(true);
- long c = (stop_counter.equals("-")) ? 0
- : job.getCounters().findCounter("mrql",stop_counter).getValue();
- return new DataSet(new BinaryDataSource(newpath,conf),c,outputRecords(job));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapOperation.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapOperation.java b/src/main/java/MapReduce/MapOperation.java
deleted file mode 100644
index b5c9a8f..0000000
--- a/src/main/java/MapReduce/MapOperation.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-
-/** The MapReduce operation that use a map stage only */
-final public class MapOperation extends MapReducePlan {
-
- /** The mapper of Map */
- private final static class cMapMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
- private static String counter; // a Hadoop user-defined counter used in the repeat operation
- private static Function map_fnc; // the mapper function
- private static Function acc_fnc; // aggregator
- private static MRData result; // aggregation result
- private static Tuple pair = new Tuple(2);
- private static MRContainer container = new MRContainer(new MR_int(0));
-
- private void write ( MRContainer key, MRData value, Context context )
- throws IOException, InterruptedException {
- if (result != null) { // aggregation
- pair.set(0,result);
- pair.set(1,value);
- result = acc_fnc.eval(pair);
- } else if (counter.equals("-")) {
- container.set(value);
- context.write(key,container);
- } else { // increment the repetition counter if the repeat condition is true
- Tuple t = (Tuple)value;
- if (((MR_bool)t.second()).get())
- context.getCounter("mrql",counter).increment(1);
- container.set(t.first());
- context.write(key,container);
- }
- }
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- for ( MRData e: (Bag)map_fnc.eval(value.data()) )
- write(key,e,context);
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.mapper"));
- map_fnc = functional_argument(conf,code);
- if (conf.get("mrql.zero") != null) {
- code = Tree.parse(conf.get("mrql.zero"));
- result = Interpreter.evalE(code);
- code = Tree.parse(conf.get("mrql.accumulator"));
- acc_fnc = functional_argument(conf,code);
- } else result = null;
- counter = conf.get("mrql.counter");
- } catch (Exception e) {
- throw new Error("Cannot retrieve the mapper plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (result != null) // emit the result of aggregation
- context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
- super.cleanup(context);
- }
- }
-
- /** The cMap physical operator
- * @param map_fnc mapper function
- * @param acc_fnc optional accumulator function
- * @param zero optional the zero value for the accumulator
- * @param source input data source
- * @param stop_counter optional counter used in repeat operation
- * @return a new data source that contains the result
- */
- public final static DataSet cMap ( Tree map_fnc, // mapper function
- Tree acc_fnc, // optional accumulator function
- Tree zero, // optional the zero value for the accumulator
- DataSet source, // input data source
- String stop_counter ) // optional counter used in repeat operation
- throws Exception {
- String newpath = new_path(conf);
- conf.set("mrql.mapper",map_fnc.toString());
- conf.set("mrql.counter",stop_counter);
- if (zero != null) {
- conf.set("mrql.accumulator",acc_fnc.toString());
- conf.set("mrql.zero",zero.toString());
- conf.set("mapred.min.split.size","268435456");
- } else conf.set("mrql.zero","");
- Job job = new Job(conf,newpath);
- distribute_compiled_arguments(job.getConfiguration());
- job.setJarByClass(MapReducePlan.class);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setOutputFormatClass(SequenceFileOutputFormat.class);
- for (DataSource p: source.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,cMapMapper.class);
- FileOutputFormat.setOutputPath(job,new Path(newpath));
- job.setNumReduceTasks(0);
- job.waitForCompletion(true);
- long c = (stop_counter.equals("-")) ? 0
- : job.getCounters().findCounter("mrql",stop_counter).getValue();
- return new DataSet(new BinaryDataSource(newpath,conf),c,outputRecords(job));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReduceBinaryInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReduceBinaryInputFormat.java b/src/main/java/MapReduce/MapReduceBinaryInputFormat.java
deleted file mode 100644
index 36ad094..0000000
--- a/src/main/java/MapReduce/MapReduceBinaryInputFormat.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-
-
-/** Input format for Apache Hadoop sequence files */
-final public class MapReduceBinaryInputFormat extends MapReduceMRQLFileInputFormat {
- final static SequenceFileInputFormat<MRContainer,MRContainer> inputFormat
- = new SequenceFileInputFormat<MRContainer,MRContainer>();
-
- public RecordReader<MRContainer,MRContainer>
- createRecordReader ( InputSplit split,
- TaskAttemptContext context ) throws IOException, InterruptedException {
- return inputFormat.createRecordReader(split,context);
- }
-
- /** collect the data from multiple sequence files at the path directory into a Bag
- * @param path the path directory
- * @return a Bag that contains all data
- */
- public Bag materialize ( final Path path ) throws IOException {
- final FileSystem fs = path.getFileSystem(Plan.conf);
- final FileStatus[] ds
- = fs.listStatus(path,
- new PathFilter () {
- public boolean accept ( Path path ) {
- return !path.getName().startsWith("_");
- }
- });
- if (ds.length > 0)
- return new Bag(new BagIterator () {
- SequenceFile.Reader reader = new SequenceFile.Reader(fs,ds[0].getPath(),Plan.conf);
- MRContainer key = new MRContainer(new MR_int(0));
- MRContainer value = new MRContainer(new MR_int(0));
- int i = 1;
- public boolean hasNext () {
- try {
- if (reader.next(key,value))
- return true;
- do {
- if (i >= ds.length)
- return false;
- reader.close();
- reader = new SequenceFile.Reader(fs,ds[i++].getPath(),Plan.conf);
- } while (!reader.next(key,value));
- return true;
- } catch (IOException e) {
- throw new Error("Cannot collect values from an intermediate result");
- }
- }
- public MRData next () {
- return value.data();
- }
- });
- return new Bag();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReduceEvaluator.gen
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReduceEvaluator.gen b/src/main/java/MapReduce/MapReduceEvaluator.gen
deleted file mode 100644
index e733715..0000000
--- a/src/main/java/MapReduce/MapReduceEvaluator.gen
+++ /dev/null
@@ -1,296 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java_cup.runtime.*;
-import org.apache.mrql.gen.*;
-import java.util.*;
-import java.io.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** Evaluates physical plans in Apache Hadoop MapReduce mode */
-final public class MapReduceEvaluator extends Evaluator {
-
- /** initialize the MapReduce evaluator */
- final public void init ( Configuration conf ) {
- Config.map_reduce_mode = true;
- if (Config.hadoop_mode)
- if (Config.local_mode) {
- conf.set("mapred.job.tracker","local");
- conf.set("fs.default.name","file:///");
- } else {
- conf.set("mapred.job.tracker",System.getenv("MAPRED_JOB_TRACKER"));
- conf.set("fs.default.name",System.getenv("FS_DEFAULT_NAME"));
- }
- }
-
- /** shutdown the MapReduce evaluator */
- final public void shutdown ( Configuration conf ) {
- }
-
- /** initialize the query evaluation */
- final public void initialize_query () {
- }
-
- /** create a new evaluation configuration */
- final public Configuration new_configuration () {
- return new Configuration();
- }
-
- /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
- final public Class<? extends MRQLFileInputFormat> parsedInputFormat () {
- return MapReduceParsedInputFormat.class;
- }
-
- /** return the FileInputFormat for binary files */
- final public Class<? extends MRQLFileInputFormat> binaryInputFormat () {
- return MapReduceBinaryInputFormat.class;
- }
-
- /** return the FileInputFormat for data generator files */
- final public Class<? extends MRQLFileInputFormat> generatorInputFormat () {
- return MapReduceGeneratorInputFormat.class;
- }
-
- /** The Aggregate physical operator
- * @param acc_fnc the accumulator function from (T,T) to T
- * @param zero the zero element of type T
- * @param plan the plan that constructs the dataset that contains the bag of values {T}
- * @param env contains bindings fro variables to values (MRData)
- * @return the aggregation result of type T
- */
- final public MRData aggregate ( Tree acc_fnc,
- Tree zero,
- Tree plan,
- Environment env ) throws Exception {
- return MapReducePlan.aggregate(acc_fnc,zero,eval(plan,env,"-"));
- }
-
- /** Evaluate a loop a fixed # of times */
- final public Tuple loop ( Tree e, Environment env ) throws Exception {
- match e {
- case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`num):
- int limit = ((MR_int)evalE(num,env)).get();
- MR_dataset[] s = new MR_dataset[vs.length()];
- for ( int i = 0; i < vs.length(); i++ )
- s[i] = new MR_dataset(eval(ss.nth(i),env,"-"));
- for ( int n = 0; n < limit; n++ ) {
- Environment nenv = env;
- for ( int i = 0; i < vs.length(); i ++ )
- nenv = new Environment(vs.nth(i).toString(),s[i],nenv);
- for ( int i = 0; i < vs.length(); i ++ )
- s[i] = new MR_dataset(eval(bs.nth(i),nenv,"-"));
- };
- return new Tuple(s);
- };
- throw new Error("Wrong Loop format");
- }
-
- /** Evaluate an MRQL physical plan using Hadoop and print tracing info
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a DataSet (stored in HDFS)
- */
- final public DataSet eval ( final Tree e,
- final Environment env,
- final String counter ) {
- if (Config.trace_execution) {
- tab_count += 3;
- System.out.println(tabs(tab_count)+print_query(e));
- };
- DataSet res = evalD(e,env,counter);
- if (Config.trace_execution)
- try {
- System.out.println(tabs(tab_count)
- +"-> "+res.take(Config.max_bag_size_print));
- tab_count -= 3;
- } catch (Exception ex) {
- throw new Error("Cannot collect the operator output: "+e);
- };
- return res;
- }
-
- /** Evaluate MRQL physical operators using Hadoop
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a DataSet (stored in HDFS)
- */
- final DataSet evalD ( final Tree e,
- final Environment env,
- final String counter ) {
- try {
- match e {
- case cMap(`f,`s):
- return MapOperation.cMap(closure(f,env),null,null,eval(s,env,"-"),counter);
- case AggregateMap(`f,`acc,`zero,`s):
- return MapOperation.cMap(closure(f,env),closure(acc,env),
- (zero.equals(#<null>))?null:zero,
- eval(s,env,"-"),counter);
- case MapReduce(`m,`r,`s,`o):
- return MapReduceOperation.mapReduce(closure(m,env),#<null>,closure(r,env),
- null,null,
- eval(s,env,"-"),
- Config.nodes,counter,
- o.equals(#<true>));
- case MapAggregateReduce(`m,`r,`acc,`zero,`s,`o):
- return MapReduceOperation.mapReduce(closure(m,env),null,closure(r,env),
- closure(acc,env),
- (zero.equals(#<null>))?null:zero,
- eval(s,env,"-"),
- Config.nodes,counter,
- o.equals(#<true>));
- case MapCombineReduce(`m,`c,`r,`s,`o):
- return MapReduceOperation.mapReduce(closure(m,env),closure(c,env),closure(r,env),
- null,null,
- eval(s,env,"-"),
- Config.nodes,counter,
- o.equals(#<true>));
- case CrossProduct(`mx,`my,`r,`x,`y):
- return CrossProductOperation.crossProduct(closure(mx,env),closure(my,env),closure(r,env),
- null,null,
- eval(x,env,"-"),
- eval(y,env,"-"),
- counter);
- case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
- return CrossProductOperation.crossProduct(closure(mx,env),closure(my,env),closure(r,env),
- closure(acc,env),
- (zero.equals(#<null>))?null:zero,
- eval(x,env,"-"),
- eval(y,env,"-"),
- counter);
- case MapReduce2(`mx,`my,`r,`x,`y,`o):
- return eval(#<MapAggregateReduce2(`mx,`my,`r,null,null,`x,`y,`o)>,env,counter);
- case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,`o):
- return JoinOperation.mapReduce2(closure(mx,env),closure(my,env),
- closure(c,env),
- closure(r,env),
- null,null,
- eval(x,env,"-"),
- eval(y,env,"-"),
- Config.nodes,counter,
- o.equals(#<true>));
- case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,`o):
- return JoinOperation.mapReduce2(closure(mx,env),closure(my,env),
- null,
- closure(r,env),closure(acc,env),
- (zero.equals(#<null>))?null:zero,
- eval(x,env,"-"),
- eval(y,env,"-"),
- Config.nodes,counter,
- o.equals(#<true>));
- case GroupByJoin(`kx,`ky,`gx,`gy,`mp,`c,`r,`x,`y,`o):
- DataSet X = eval(x,env,"-");
- DataSet Y = eval(y,env,"-");
- int n = (int)Math.floor(Math.sqrt(4.0*Config.nodes));
- int m = n;
- if (Config.trace)
- System.out.println("*** Using a groupBy join on a "+n+"*"+m+" grid of partitions");
- return GroupByJoinPlan.groupByJoin(closure(kx,env),closure(ky,env),
- closure(gx,env),closure(gy,env),
- closure(mp,env),closure(c,env),closure(r,env),
- X,Y,
- Config.nodes,
- n,m,counter);
- case MapJoin(`mx,`my,`r,`x,`y):
- return MapJoinOperation.mapJoin(closure(mx,env),closure(my,env),closure(r,env),
- null,null,
- eval(x,env,"-"),
- eval(y,env,"-"),
- counter);
- case MapAggregateJoin(`mx,`my,`r,`acc,`zero,`x,`y):
- return MapJoinOperation.mapJoin(closure(mx,env),closure(my,env),closure(r,env),
- closure(acc,env),
- (zero.equals(#<null>))?null:zero,
- eval(x,env,"-"),
- eval(y,env,"-"),
- counter);
- case BinarySource(`file,_):
- return Plan.binarySource(file.stringValue());
- case ParsedSource(`parser,`file,...args):
- Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- return Plan.parsedSource(p,((MR_string)evalE(file,env)).get(),args);
- case Merge(`x,`y):
- return Plan.merge(eval(x,env,"-"),eval(y,env,"-"));
- case Repeat(lambda(`v,`b),`s,`n):
- final String nm = v.toString();
- final Tree body = b;
- Function loop_fnc = new Function () {
- public MRData eval ( MRData s ) {
- return new MR_dataset(evaluator.eval(body,new Environment(nm,s,env),nm));
- }; };
- return MapReducePlan.repeat(loop_fnc,eval(s,env,"-"),((MR_int)evalE(n,env)).get());
- case Closure(lambda(`v,`b),`s,`n):
- final String nm = v.toString();
- final Tree body = b;
- Function loop_fnc = new Function () {
- public MRData eval ( MRData s ) {
- return new MR_dataset(evaluator.eval(body,new Environment(nm,s,env),"-"));
- }; };
- return MapReducePlan.closure(loop_fnc,eval(s,env,"-"),((MR_int)evalE(n,env)).get());
- case Generator(`min,`max,`size):
- return Plan.generator(((MR_long)evalE(min,env)).get(),
- ((MR_long)evalE(max,env)).get(),
- ((MR_long)evalE(size,env)).get());
- case let(`v,`u,`body):
- return eval(body,new Environment(v.toString(),evalE(u,env),env),"-");
- case Let(`v,`u,`body):
- return eval(body,new Environment(v.toString(),new MR_dataset(eval(u,env,"-")),env),"-");
- case If(`c,`x,`y):
- if (((MR_bool)evalE(c,env)).get())
- return eval(x,env,"-");
- else return eval(y,env,"-");
- case apply(`f,`arg):
- if (!f.is_variable())
- return ((MR_dataset)evalF(f,env).eval(evalE(arg))).dataset();
- MRData fnc = variable_lookup(f.toString(),global_env);
- if (fnc == null)
- throw new Error("Unknown function: "+f);
- MRData t = evalE(arg,env);
- if (!(t instanceof Tuple))
- throw new Error("Expected a tuple in function application: "+t);
- return ((MR_dataset)((Lambda)fnc).lambda().eval(t)).dataset();
- case `v:
- if (!v.is_variable())
- fail;
- MRData x = variable_lookup(v.toString(),env);
- if (x != null)
- if (x instanceof MR_dataset)
- return ((MR_dataset)x).dataset();
- x = variable_lookup(v.toString(),global_env);
- if (x != null)
- if (x instanceof MR_dataset)
- return ((MR_dataset)x).dataset();
- throw new Error("Variable "+v+" is not bound");
- };
- throw new Error("Cannot evaluate the map-reduce plan: "+e);
- } catch (Error msg) {
- if (!Config.trace)
- throw new Error(msg.getMessage());
- System.err.println(msg.getMessage());
- throw new Error("Evaluation error in: "+print_query(e));
- } catch (Exception ex) {
- System.err.println(ex.getMessage());
- ex.printStackTrace();
- throw new Error("Evaluation error in: "+print_query(e));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReduceGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReduceGeneratorInputFormat.java b/src/main/java/MapReduce/MapReduceGeneratorInputFormat.java
deleted file mode 100644
index 9a8ec55..0000000
--- a/src/main/java/MapReduce/MapReduceGeneratorInputFormat.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-
-
-/** the FileInputFormat for data generators: it creates HDFS files, where each file contains
- * an (offset,size) pair that generates the range of values [offset,offset+size] */
-final public class MapReduceGeneratorInputFormat extends MapReduceMRQLFileInputFormat {
- public static class GeneratorRecordReader extends RecordReader<MRContainer,MRContainer> {
- final long offset;
- final long size;
- long index;
- SequenceFile.Reader reader;
-
- public GeneratorRecordReader ( FileSplit split,
- TaskAttemptContext context ) throws IOException {
- Configuration conf = context.getConfiguration();
- Path path = split.getPath();
- FileSystem fs = path.getFileSystem(conf);
- reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
- MRContainer key = new MRContainer();
- MRContainer value = new MRContainer();
- reader.next(key,value);
- offset = ((MR_long)((Tuple)(value.data())).first()).get();
- size = ((MR_long)((Tuple)(value.data())).second()).get();
- index = -1;
- }
-
- public boolean nextKeyValue () throws IOException {
- index++;
- return index < size;
- }
-
- public MRContainer getCurrentKey () throws IOException {
- return new MRContainer(new MR_long(index));
- }
-
- public MRContainer getCurrentValue () throws IOException {
- return new MRContainer(new MR_long(offset+index));
- }
-
- public void close () throws IOException { reader.close(); }
-
- public float getProgress () throws IOException {
- return index / (float)size;
- }
-
- public void initialize ( InputSplit split, TaskAttemptContext context ) throws IOException { }
- }
-
- public RecordReader<MRContainer,MRContainer>
- createRecordReader ( InputSplit split, TaskAttemptContext context ) throws IOException {
- return new GeneratorRecordReader((FileSplit)split,context);
- }
-
- /** Insert all results from the generators stored in path into a Bag.
- * The Bag is lazily constructed.
- * @param path the path directory that contains the generator data (offset,size)
- * @return a Bag that contains all data
- */
- public Bag materialize ( final Path path ) throws IOException {
- Configuration conf = Plan.conf;
- FileSystem fs = path.getFileSystem(conf);
- final SequenceFile.Reader reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
- final MRContainer key = new MRContainer();
- final MRContainer value = new MRContainer();
- return new Bag(new BagIterator () {
- long offset = 0;
- long size = 0;
- long i = 0;
- public boolean hasNext () {
- if (++i >= offset+size)
- try {
- if (!reader.next(key,value))
- return false;
- offset = ((MR_long)((Tuple)(value.data())).first()).get();
- size = ((MR_long)((Tuple)(value.data())).second()).get();
- i = offset;
- } catch (IOException e) {
- throw new Error("Cannot collect values from a generator");
- };
- return true;
- }
- public MRData next () {
- return new MR_long(i);
- }
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReduceMRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReduceMRQLFileInputFormat.java b/src/main/java/MapReduce/MapReduceMRQLFileInputFormat.java
deleted file mode 100644
index d0760b4..0000000
--- a/src/main/java/MapReduce/MapReduceMRQLFileInputFormat.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-
-
-/** A superclass for all MRQL FileInputFormats */
-abstract public class MapReduceMRQLFileInputFormat extends FileInputFormat<MRContainer,MRContainer> implements MRQLFileInputFormat {
- public MapReduceMRQLFileInputFormat () {}
-
- /** record reader for map-reduce */
- abstract public RecordReader<MRContainer,MRContainer>
- createRecordReader ( InputSplit split,
- TaskAttemptContext context ) throws IOException, InterruptedException;
-
- /** materialize the input file into a memory Bag */
- abstract public Bag materialize ( final Path path ) throws IOException;
-
- /** materialize the entire dataset into a Bag
- * @param x the DataSet in HDFS to collect values from
- * @param strip is not used in MapReduce mode
- * @return the Bag that contains the collected values
- */
- public final Bag collect ( final DataSet x, boolean strip ) throws Exception {
- Bag res = new Bag();
- for ( DataSource s: x.source )
- if (s.to_be_merged)
- res = res.union(Plan.merge(s));
- else res = res.union(s.inputFormat.newInstance().materialize(new Path(s.path)));
- return res;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReduceOperation.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReduceOperation.java b/src/main/java/MapReduce/MapReduceOperation.java
deleted file mode 100644
index 39864b3..0000000
--- a/src/main/java/MapReduce/MapReduceOperation.java
+++ /dev/null
@@ -1,273 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.Hashtable;
-import java.util.Iterator;
-import java.util.Enumeration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-
-
-/** The MapReduce operation that uses an in-mapper combiner to partially reduce groups during mapping */
-final public class MapReduceOperation extends MapReducePlan {
-
- public final static class MRContainerPartitioner extends Partitioner<MRContainer,MRContainer> {
- final public int getPartition ( MRContainer key, MRContainer value, int numPartitions ) {
- return Math.abs(key.hashCode()) % numPartitions;
- }
- }
-
- /** The mapper of the MapReduce operation */
- private final static class MRMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
- private static Function map_fnc; // the map function
- private static Function combine_fnc; // the combiner function
- private static Hashtable<MRData,MRData> hashTable; // in-mapper combiner
- private static int index;
- private static Tuple tkey = new Tuple(2);
- private static Bag tbag = new Bag(2);
- private static MRContainer ckey = new MRContainer(new MR_int(0));
- private static MRContainer cvalue = new MRContainer(new MR_int(0));
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- for ( MRData e: (Bag)map_fnc.eval(value.data()) ) {
- Tuple p = (Tuple)e;
- if (hashTable == null) {
- ckey.set(p.first());
- cvalue.set(p.second());
- context.write(ckey,cvalue);
- } else {
- // in-mapper combiner
- MRData old = hashTable.get(p.first());
- if (old == null) {
- if (index++ == Config.map_cache_size)
- flush_table(context);
- hashTable.put(p.first(),p.second());
- } else {
- tkey.set(0,p.first());
- tbag.clear();
- tbag.add_element(p.second()).add_element(old);
- tkey.set(1,tbag);
- for ( MRData x: (Bag)combine_fnc.eval(tkey) )
- hashTable.put(p.first(),x); // normally, done once
- }
- }
- }
- }
-
- private static void flush_table ( Context context ) throws IOException, InterruptedException {
- Enumeration<MRData> en = hashTable.keys();
- while (en.hasMoreElements()) {
- MRData key = en.nextElement();
- ckey.set(key);
- MRData value = hashTable.get(key);
- cvalue.set(value);
- if (value != null)
- context.write(ckey,cvalue);
- };
- index = 0;
- hashTable.clear();
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.mapper"));
- map_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.combiner"));
- hashTable = null;
- if (code != null && !code.equals(new VariableLeaf("null"))) {
- combine_fnc = functional_argument(conf,code);
- hashTable = new Hashtable<MRData,MRData>(Config.map_cache_size);
- index = 0;
- }
- } catch (Exception e) {
- throw new Error("Cannot retrieve the mapper plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (hashTable != null)
- flush_table(context);
- hashTable = null; // garbage-collect it
- super.cleanup(context);
- }
- }
-
- /** The reducer of the MapReduce operation */
- private final static class MRReducer extends Reducer<MRContainer,MRContainer,MRContainer,MRContainer> {
- private static String counter; // a Hadoop user-defined counter used in the repeat operation
- private static Function reduce_fnc; // the reduce function
- private static Bag s = new Bag(); // a cached bag of input fragments
- private static Function acc_fnc; // aggregator
- private static MRData result; // aggregation result
- private static boolean streamed = false;
- private static Tuple pair = new Tuple(2);
- private static MRContainer container = new MRContainer(new MR_int(0));
-
- private void write ( MRContainer key, MRData value, Context context )
- throws IOException, InterruptedException {
- if (result != null) { // aggregation
- pair.set(0,result);
- pair.set(1,value);
- result = acc_fnc.eval(pair);
- } else if (counter.equals("-")) {
- container.set(value);
- context.write(key,container);
- } else { // increment the repetition counter if the repeat condition is true
- Tuple t = (Tuple)value;
- if (((MR_bool)t.second()).get())
- context.getCounter("mrql",counter).increment(1);
- container.set(t.first());
- context.write(key,container);
- }
- }
-
- @Override
- public void reduce ( MRContainer key, Iterable<MRContainer> values, Context context )
- throws IOException, InterruptedException {
- if (!streamed) { // store the values in a Bag and then reduce
- s.clear();
- for ( MRContainer val: values )
- s.add(val.data());
- pair.set(0,key.data());
- pair.set(1,s);
- for ( MRData e: (Bag)reduce_fnc.eval(pair) )
- write(key,e,context);
- } else { // it accesses the values in stream-like fashion
- final Iterator<MRContainer> iterator = values.iterator();
- Bag s = new Bag(new BagIterator() {
- public boolean hasNext () {
- return iterator.hasNext();
- }
- public MRData next () {
- return iterator.next().data();
- }
- });
- pair.set(0,key.data());
- pair.set(1,s);
- for ( MRData e: (Bag)reduce_fnc.eval(pair) )
- write(key,e,context);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.reducer"));
- reduce_fnc = functional_argument(conf,code);
- streamed = PlanGeneration.streamed_MapReduce_reducer(code);
- if (conf.get("mrql.zero") != null) {
- code = Tree.parse(conf.get("mrql.zero"));
- result = Interpreter.evalE(code);
- code = Tree.parse(conf.get("mrql.accumulator"));
- acc_fnc = functional_argument(conf,code);
- } else result = null;
- counter = conf.get("mrql.counter");
- } catch (Exception e) {
- throw new Error("Cannot retrieve the reduce plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (result != null) // emit the result of aggregation
- context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
- super.cleanup(context);
- }
- }
-
- /**
- * The MapReduce physical operator
- * @param map_fnc the mapper function
- * @param combine_fnc optional in-mapper combiner function
- * @param reduce_fnc the reducer function
- * @param acc_fnc optional accumulator function
- * @param zero optional the zero value for the accumulator
- * @param source the input data source
- * @param num_reduces number of reducers
- * @param stop_counter optional counter used in repeat operation
- * @param orderp does the result need to be ordered?
- * @return a new data source that contains the result
- */
- public final static DataSet mapReduce ( Tree map_fnc, // mapper function
- Tree combine_fnc, // optional in-mapper combiner function
- Tree reduce_fnc, // reducer function
- Tree acc_fnc, // optional accumulator function
- Tree zero, // optional the zero value for the accumulator
- DataSet source, // input data source
- int num_reduces, // number of reducers
- String stop_counter, // optional counter used in repeat operation
- boolean orderp ) // does the result need to be ordered?
- throws Exception {
- String newpath = new_path(conf);
- conf.set("mrql.mapper",map_fnc.toString());
- if (combine_fnc != null)
- conf.set("mrql.combiner",combine_fnc.toString());
- conf.set("mrql.reducer",reduce_fnc.toString());
- if (zero != null) { // will use in-mapper combiner
- conf.set("mrql.accumulator",acc_fnc.toString());
- conf.set("mrql.zero",zero.toString());
- // the in-mapper combiner likes large data splits
- conf.set("mapred.min.split.size","268435456"); // 256 MBs
- } else conf.set("mrql.zero","");
- conf.set("mrql.counter",stop_counter);
- Job job = new Job(conf,newpath);
- distribute_compiled_arguments(job.getConfiguration());
- job.setJarByClass(MapReducePlan.class);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setPartitionerClass(MRContainerPartitioner.class);
- job.setSortComparatorClass(MRContainerKeyComparator.class);
- job.setGroupingComparatorClass(MRContainerKeyComparator.class);
- job.setOutputFormatClass(SequenceFileOutputFormat.class);
- for (DataSource p: source.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MRMapper.class);
- FileOutputFormat.setOutputPath(job,new Path(newpath));
- job.setReducerClass(MRReducer.class);
- if (Config.trace && PlanGeneration.streamed_MapReduce_reducer(reduce_fnc))
- System.err.println("*** Streamed MapReduce reducer");
- if (num_reduces > 0)
- job.setNumReduceTasks(num_reduces);
- job.waitForCompletion(true);
- long c = (stop_counter.equals("-")) ? 0
- : job.getCounters().findCounter("mrql",stop_counter).getValue();
- DataSource s = new BinaryDataSource(newpath,conf);
- s.to_be_merged = orderp;
- return new DataSet(s,c,outputRecords(job));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReduceParsedInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReduceParsedInputFormat.java b/src/main/java/MapReduce/MapReduceParsedInputFormat.java
deleted file mode 100644
index e588dbd..0000000
--- a/src/main/java/MapReduce/MapReduceParsedInputFormat.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.Trees;
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-
-
-/** A FileInputFormat for text files (CVS, XML, JSON, ...) */
-final public class MapReduceParsedInputFormat extends MapReduceMRQLFileInputFormat {
-
- public static class ParsedRecordReader extends RecordReader<MRContainer,MRContainer> {
- final FSDataInputStream fsin;
- final long start;
- final long end;
- Iterator<MRData> result;
- MRData data;
- Parser parser;
-
- public ParsedRecordReader ( FileSplit split,
- TaskAttemptContext context,
- Class<? extends Parser> parser_class,
- Trees args ) throws IOException {
- Configuration conf = context.getConfiguration();
- start = split.getStart();
- end = start + split.getLength();
- Path file = split.getPath();
- FileSystem fs = file.getFileSystem(conf);
- fsin = fs.open(split.getPath());
- try {
- parser = parser_class.newInstance();
- } catch (Exception ex) {
- throw new Error("Unrecognized parser:"+parser_class);
- };
- parser.initialize(args);
- parser.open(fsin,start,end);
- result = null;
- }
-
- public boolean nextKeyValue () throws IOException {
- while (result == null || !result.hasNext()) {
- String s = parser.slice();
- if (s == null)
- return false;
- result = parser.parse(s).iterator();
- };
- data = (MRData)result.next();
- return true;
- }
-
- public MRContainer getCurrentKey () throws IOException {
- return new MRContainer(new MR_long(fsin.getPos()));
- }
-
- public MRContainer getCurrentValue () throws IOException {
- return new MRContainer(data);
- }
-
- public long getPos () throws IOException { return fsin.getPos(); }
-
- public void close () throws IOException { fsin.close(); }
-
- public float getProgress () throws IOException {
- if (end == start)
- return 0.0f;
- else return Math.min(1.0f, (getPos() - start) / (float)(end - start));
- }
-
- public void initialize ( InputSplit split, TaskAttemptContext context ) throws IOException { }
- }
-
- public RecordReader<MRContainer,MRContainer>
- createRecordReader ( InputSplit split,
- TaskAttemptContext context ) throws IOException, InterruptedException {
- Configuration conf = context.getConfiguration();
- String path = ((FileSplit)split).getPath().toString();
- ParsedDataSource ds = (ParsedDataSource)DataSource.get(path,conf);
- return new ParsedRecordReader((FileSplit)split,context,ds.parser,(Trees)ds.args);
- }
-
- /** Find the parser associated with each file in the path and parse the file,
- * inserting all results into a Bag. The Bag is lazily constructed.
- * @param path the path directory with the files
- * @return a Bag that contains all data
- */
- public Bag materialize ( final Path path ) throws IOException {
- Configuration conf = Plan.conf;
- ParsedDataSource ds = (ParsedDataSource)DataSource.get(path.toString(),conf);
- FileSystem fs = path.getFileSystem(conf);
- FSDataInputStream fsin = fs.open(path);
- Parser p;
- try {
- p = ds.parser.newInstance();
- } catch (Exception ex) {
- throw new Error("Unrecognized parser:"+ds.parser);
- };
- final Parser parser = p;
- parser.initialize(ds.args);
- parser.open(fsin,0,Long.MAX_VALUE);
- return new Bag(new BagIterator () {
- Iterator<MRData> iter;
- public boolean hasNext () {
- while (iter == null || !iter.hasNext()) {
- String line = parser.slice();
- if (line == null)
- return false;
- iter = parser.parse(line).iterator();
- };
- return true;
- }
- public MRData next () {
- return iter.next();
- }
- });
- }
-}
[13/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/BSPGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/BSPGeneratorInputFormat.java b/src/main/java/BSP/BSPGeneratorInputFormat.java
deleted file mode 100644
index bdee64e..0000000
--- a/src/main/java/BSP/BSPGeneratorInputFormat.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.Trees;
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hama.bsp.*;
-
-
-/** the FileInputFormat for data generators: it creates HDFS files, where each file contains
- * an (offset,size) pair that generates the range of values [offset,offset+size] */
-final public class BSPGeneratorInputFormat extends BSPMRQLFileInputFormat {
- public static class GeneratorRecordReader implements RecordReader<MRContainer,MRContainer> {
- final long offset;
- final long size;
- final int source_number;
- final MRData source_num_data;
- long index;
- SequenceFile.Reader reader;
-
- public GeneratorRecordReader ( FileSplit split,
- int source_number,
- BSPJob job ) throws IOException {
- Configuration conf = BSPPlan.getConfiguration(job);
- Path path = split.getPath();
- FileSystem fs = path.getFileSystem(conf);
- reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
- MRContainer key = new MRContainer();
- MRContainer value = new MRContainer();
- reader.next(key,value);
- offset = ((MR_long)((Tuple)(value.data())).first()).get();
- size = ((MR_long)((Tuple)(value.data())).second()).get();
- this.source_number = source_number;
- source_num_data = new MR_int(source_number);
- index = -1;
- }
-
- public MRContainer createKey () {
- return new MRContainer(null);
- }
-
- public MRContainer createValue () {
- return new MRContainer(null);
- }
-
- public boolean next ( MRContainer key, MRContainer value ) throws IOException {
- index++;
- value.set(new Tuple(source_num_data,new MR_long(offset+index)));
- key.set(new MR_long(index));
- return index < size;
- }
-
- public long getPos () throws IOException { return index; }
-
- public void close () throws IOException { reader.close(); }
-
- public float getProgress () throws IOException {
- return index / (float)size;
- }
-
- public void initialize ( InputSplit split, TaskAttemptContext context ) throws IOException { }
- }
-
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split, BSPJob job ) throws IOException {
- Configuration conf = BSPPlan.getConfiguration(job);
- String path = ((FileSplit)split).getPath().toString();
- GeneratorDataSource ds = (GeneratorDataSource)DataSource.get(path,conf);
- return new GeneratorRecordReader((FileSplit)split,ds.source_num,job);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/BSPMRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/BSPMRQLFileInputFormat.java b/src/main/java/BSP/BSPMRQLFileInputFormat.java
deleted file mode 100644
index 71518cd..0000000
--- a/src/main/java/BSP/BSPMRQLFileInputFormat.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hama.bsp.*;
-import org.apache.hama.HamaConfiguration;
-
-
-/** A superclass for all MRQL FileInputFormats */
-abstract public class BSPMRQLFileInputFormat extends FileInputFormat<MRContainer,MRContainer> implements MRQLFileInputFormat {
- public BSPMRQLFileInputFormat () {}
-
- /** record reader for map-reduce */
- abstract public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split, BSPJob job ) throws IOException;
-
- /** materialize the input file into a memory Bag */
- public Bag materialize ( final Path file ) throws IOException {
- final BSPJob job = new BSPJob((HamaConfiguration)Plan.conf,MRQLFileInputFormat.class);
- job.setInputPath(file);
- final InputSplit[] splits = getSplits(job,1);
- final RecordReader<MRContainer,MRContainer> rd = getRecordReader(splits[0],job);
- return new Bag(new BagIterator () {
- RecordReader<MRContainer,MRContainer> reader = rd;
- MRContainer key = reader.createKey();
- MRContainer value = reader.createKey();
- int i = 0;
- public boolean hasNext () {
- try {
- if (reader.next(key,value))
- return true;
- do {
- if (++i >= splits.length)
- return false;
- reader.close();
- reader = getRecordReader(splits[i],job);
- } while (!reader.next(key,value));
- return true;
- } catch (IOException e) {
- throw new Error("Cannot collect values from an intermediate result");
- }
- }
- public MRData next () {
- return value.data();
- }
- });
- }
-
- /** materialize the entire dataset into a Bag
- * @param x the DataSet in HDFS to collect values from
- * @param strip true if you want to stripout the source id (used in BSP sources)
- * @return the Bag that contains the collected values
- */
- public final Bag collect ( final DataSet x, boolean strip ) throws Exception {
- Bag res = new Bag();
- for ( DataSource s: x.source )
- if (s.to_be_merged)
- res = res.union(Plan.merge(s));
- else {
- Path path = new Path(s.path);
- final FileSystem fs = path.getFileSystem(Plan.conf);
- final FileStatus[] ds
- = fs.listStatus(path,
- new PathFilter () {
- public boolean accept ( Path path ) {
- return !path.getName().startsWith("_");
- }
- });
- Bag b = new Bag();
- for ( FileStatus st: ds )
- b = b.union(s.inputFormat.newInstance().materialize(st.getPath()));
- if (strip) {
- // remove source_num
- final Iterator<MRData> iter = b.iterator();
- b = new Bag(new BagIterator() {
- public boolean hasNext () {
- return iter.hasNext();
- }
- public MRData next () {
- return ((Tuple)iter.next()).get(1);
- }
- });
- };
- res = res.union(b);
- };
- return res;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/BSPParsedInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/BSPParsedInputFormat.java b/src/main/java/BSP/BSPParsedInputFormat.java
deleted file mode 100644
index 7da9a2e..0000000
--- a/src/main/java/BSP/BSPParsedInputFormat.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.Trees;
-import java.io.*;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hama.bsp.*;
-
-
-/** A FileInputFormat for text files (CVS, XML, JSON, ...) */
-final public class BSPParsedInputFormat extends BSPMRQLFileInputFormat {
- public static class ParsedRecordReader implements RecordReader<MRContainer,MRContainer> {
- final FSDataInputStream fsin;
- final long start;
- final long end;
- final int source_number;
- final MRData source_num_data;
- Iterator<MRData> result;
- Parser parser;
-
- public ParsedRecordReader ( FileSplit split,
- BSPJob job,
- Class<? extends Parser> parser_class,
- int source_number,
- Trees args ) throws IOException {
- Configuration conf = BSPPlan.getConfiguration(job);
- start = split.getStart();
- end = start + split.getLength();
- Path file = split.getPath();
- FileSystem fs = file.getFileSystem(conf);
- fsin = fs.open(split.getPath());
- try {
- parser = parser_class.newInstance();
- } catch (Exception ex) {
- throw new Error("Unrecognized parser:"+parser_class);
- };
- this.source_number = source_number;
- source_num_data = new MR_int(source_number);
- parser.initialize(args);
- parser.open(fsin,start,end);
- result = null;
- }
-
- public MRContainer createKey () {
- return new MRContainer();
- }
-
- public MRContainer createValue () {
- return new MRContainer();
- }
-
- public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
- while (result == null || !result.hasNext()) {
- String s = parser.slice();
- if (s == null)
- return false;
- result = parser.parse(s).iterator();
- };
- value.set(new Tuple(source_num_data,(MRData)result.next()));
- key.set(new MR_long(fsin.getPos()));
- return true;
- }
-
- public synchronized long getPos () throws IOException { return fsin.getPos(); }
-
- public synchronized void close () throws IOException { fsin.close(); }
-
- public float getProgress () throws IOException {
- if (end == start)
- return 0.0f;
- else return Math.min(1.0f, (getPos() - start) / (float)(end - start));
- }
- }
-
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split,
- BSPJob job ) throws IOException {
- Configuration conf = BSPPlan.getConfiguration(job);
- String path = ((FileSplit)split).getPath().toString();
- ParsedDataSource ds = (ParsedDataSource)DataSource.get(path,conf);
- return new ParsedRecordReader((FileSplit)split,job,ds.parser,ds.source_num,(Trees)ds.args);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/BSPPlan.java
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/BSPPlan.java b/src/main/java/BSP/BSPPlan.java
deleted file mode 100644
index bca8ab9..0000000
--- a/src/main/java/BSP/BSPPlan.java
+++ /dev/null
@@ -1,507 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.Arrays;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hama.bsp.*;
-import org.apache.hama.bsp.sync.SyncException;
-import org.apache.hama.HamaConfiguration;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** Evaluate a BSP plan using Hama */
-final public class BSPPlan extends Plan {
-
- final static Configuration getConfiguration ( BSPJob job ) {
- return job.getConf(); // use job.getConfiguration() for Hama 0.6.0
- }
-
- /** The BSP evaluator */
- final static class BSPop extends BSP<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> {
- final static MRContainer null_key = new MRContainer(new MR_byte(0));
- // special message for sub-sync()
- final static MRData more_to_come = new MR_sync();
- final static MRData more_supersteps = new MR_more_bsp_steps();
-
- private int source_num;
- private Function superstep_fnc; // superstep function
- private MRData state; // BSP state
- private boolean orderp; // will output be ordered?
- private MRData source; // BSP input
- private Function acc_fnc; // aggregator
- private MRData acc_result; // aggregation result
- private static String[] all_peer_names; // all BSP peer names
- private static BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer>[] all_peers; // all BSP peers
- // a master peer that coordinates and collects results of partial aggregations
- private String masterTask;
- // buffer for received messages -- regularly in a vector, but can be spilled in a local file
- Bag msg_cache;
- // the cache that holds all local data in memory
- Tuple local_cache;
-
- private static BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> getPeer ( String name ) {
- for ( int i = 0; i < all_peer_names.length; i++ )
- if (all_peer_names[i].equals(name))
- return all_peers[i];
- throw new Error("Unknown peer: "+name);
- }
-
- private static void setPeer ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer ) {
- String name = peer.getPeerName();
- for ( int i = 0; i < all_peer_names.length; i++ )
- if (all_peer_names[i].equals(name))
- all_peers[i] = peer;
- }
-
- /** shuffle values to BSP peers based on uniform hashing on key */
- private static String shuffle ( MRData key ) {
- return all_peer_names[Math.abs(key.hashCode()) % all_peer_names.length];
- }
-
- /** to exit a BSP loop, all peers must agree to exit (this is used in BSPTranslate.bspSimplify) */
- public static MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
- return synchronize(getPeer(peerName.get()),mr_exit);
- }
-
- /** to exit a BSP loop, all peers must agree to exit (this is used in BSPTranslate.bspSimplify) */
- public static MR_bool synchronize ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer, MR_bool mr_exit ) {
- if (!Config.hadoop_mode)
- return mr_exit;
- // shortcut: if we know for sure that all peers want to exit/continue, we don't have to poll
- if (mr_exit == SystemFunctions.bsp_true_value // must be ==, not equals
- || mr_exit == SystemFunctions.bsp_false_value)
- return mr_exit;
- try {
- // this case is only used for checking the exit condition of repeat/closure
- boolean exit = mr_exit.get();
- if (all_peer_names.length <= 1)
- return (exit) ? SystemFunctions.bsp_true_value : SystemFunctions.bsp_false_value;
- if (!exit)
- // this peer is not ready to exit, so no peer should exit
- for ( String p: peer.getAllPeerNames() )
- peer.send(p,new MRContainer(more_supersteps));
- peer.sync();
- // now exit is true if no peer sent a "more_supersteps" message
- exit = peer.getNumCurrentMessages() == 0;
- peer.clear();
- return (exit) ? SystemFunctions.bsp_true_value : SystemFunctions.bsp_false_value;
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
-
- /** collect a bag from all peers by distributing the local copy s */
- public static Bag distribute ( MR_string peerName, Bag s ) {
- BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer = getPeer(peerName.get());
- if (!Config.hadoop_mode)
- return s;
- try {
- for ( MRData e: s )
- for ( String p: all_peer_names )
- peer.send(p,new MRContainer(e));
- peer.sync();
- MRContainer msg;
- Bag res = new Bag();
- while ((msg = peer.getCurrentMessage()) != null)
- if (!res.contains(msg.data()))
- res.add(msg.data());
- peer.clear();
- return res;
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
-
- private void readLocalSources ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
- throws IOException {
- MRContainer key = new MRContainer();
- MRContainer value = new MRContainer();
- while (peer.readNext(key,value)) {
- Tuple p = (Tuple)(value.data());
- ((Bag)local_cache.get(((MR_int)p.first()).get())).add(p.second());
- }
- }
-
- private void writeLocalResult ( Bag result,
- BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
- throws IOException {
- MRContainer key = new MRContainer();
- MRContainer value = new MRContainer();
- for ( MRData v: result )
- if (orderp) { // prepare for sorting
- Tuple t = (Tuple)v;
- key.set(t.get(1));
- value.set(t.get(0));
- peer.write(key,value);
- } else {
- value.set(v);
- peer.write(null_key,value);
- }
- }
-
- /** receive messages from other peers */
- private void receive_messages ( final BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
- throws IOException, SyncException, InterruptedException {
- if (Config.bsp_msg_size <= 0) { // no buffering
- msg_cache = new Bag(new BagIterator() {
- MRContainer msg;
- public boolean hasNext () {
- try {
- return (msg = peer.getCurrentMessage()) != null;
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
- public MRData next () {
- return msg.data();
- }
- });
- } else {
- boolean expect_more = false; // are we expecting more incoming messages?
- do {
- // just in case this peer did a regular-sync() before the others did a sub-sync()
- expect_more = false;
- MRContainer msg;
- // cache the received messages
- while ((msg = peer.getCurrentMessage()) != null)
- // if at least one peer sends a more_to_come message, then expect_more
- if (msg.data().equals(more_to_come))
- expect_more = true;
- else msg_cache.add(msg.data());
- if (expect_more)
- peer.sync(); // sub-sync()
- } while (expect_more);
- }
- }
-
- /** send the messages produced by a superstep to peers and then receive the replies */
- private void send_messages ( Bag msgs,
- BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
- throws IOException, SyncException, InterruptedException {
- int size = 0;
- if (Config.bsp_msg_size > 0)
- msg_cache.clear();
- for ( MRData m: msgs ) {
- Tuple t = (Tuple)m;
- // if there are too many messages to send, then sub-sync()
- if ( Config.bsp_msg_size > 0 && size++ > Config.bsp_msg_size ) {
- // tell all peers that there is more to come after sync
- for ( String p: all_peer_names )
- if (!peer.getPeerName().equals(p))
- peer.send(p,new MRContainer(more_to_come));
- peer.sync(); // sub-sync()
- size = 0;
- MRContainer msg;
- // cache the received messages
- while ((msg = peer.getCurrentMessage()) != null)
- if (!msg.data().equals(more_to_come))
- msg_cache.add(msg.data());
- };
- // suffle messages based on key; needs new MRContainer object
- peer.send(shuffle(t.get(0)),new MRContainer(t.get(1)));
- };
- peer.sync(); // regular-sync()
- receive_messages(peer);
- }
-
- @Override
- public void bsp ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer )
- throws IOException, SyncException, InterruptedException {
- final Tuple stepin = new Tuple(4);
- stepin.set(3,new MR_string(peer.getPeerName()));
- Tuple result;
- boolean skip = false;
- String tabs = "";
- int step = 0;
- boolean exit;
- if (Evaluator.evaluator == null)
- try {
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
- } catch (Exception ex) {
- throw new Error(ex);
- };
- readLocalSources(peer);
- setPeer(peer);
- do {
- if (!skip)
- step++;
- if (!skip && Config.trace_execution) {
- tabs = Interpreter.tabs(Interpreter.tab_count);
- System.err.println(tabs+" Superstep "+step+" ["+peer.getPeerName()+"]:");
- System.err.println(tabs+" messages ["+peer.getPeerName()+"]: "+msg_cache);
- System.err.println(tabs+" state ["+peer.getPeerName()+"]: "+state);
- for ( int i = 0; i < local_cache.size(); i++)
- if (local_cache.get(i) instanceof Bag && ((Bag)local_cache.get(i)).size() > 0)
- System.out.println(tabs+" cache ["+peer.getPeerName()+"] "+i+": "+local_cache.get(i));
- };
- stepin.set(0,local_cache);
- stepin.set(1,msg_cache);
- stepin.set(2,state);
- // evaluate one superstep
- result = (Tuple)superstep_fnc.eval(stepin);
- Bag msgs = (Bag)result.get(0);
- exit = ((MR_bool)result.get(2)).get();
- state = result.get(1);
- // shortcuts: if we know for sure that all peers want to exit/continue
- if (result.get(2) == SystemFunctions.bsp_true_value) { // must be ==, not equals
- peer.sync();
- if (Config.trace_execution)
- System.err.println(tabs+" result ["+peer.getPeerName()+"]: "+result);
- break;
- };
- if (result.get(2) == SystemFunctions.bsp_false_value) {
- if (Config.trace_execution)
- System.err.println(tabs+" result ["+peer.getPeerName()+"]: "+result);
- send_messages(msgs,peer);
- skip = false;
- continue;
- };
- // shortcut: skip is true when NONE of the peers sent any messages
- skip = (msgs == SystemFunctions.bsp_empty_bag); // must be ==, not equals
- if (skip)
- continue;
- if (Config.trace_execution)
- System.err.println(tabs+" result ["+peer.getPeerName()+"]: "+result);
- exit = synchronize(peer,(MR_bool)result.get(2)).get();
- send_messages(msgs,peer);
- } while (!exit);
- if (acc_result == null) {
- // the BSP result is a bag that needs to be dumped to the HDFS
- writeLocalResult((Bag)(local_cache.get(source_num)),peer);
- // if there more results, dump them to HDFS
- final MR_long key = new MR_long(0);
- final MRContainer key_container = new MRContainer(key);
- final MRContainer data_container = new MRContainer(new MR_int(0));
- int loc = 0;
- while ( loc < all_peer_names.length && peer.getPeerName().equals(all_peer_names[loc]) )
- loc++;
- Configuration conf = peer.getConfiguration();
- String[] out_paths = conf.get("mrql.output.paths").split(",");
- for ( int i = 1; i < out_paths.length; i++ ) {
- String[] s = out_paths[i].split(":");
- int out_num = Integer.parseInt(s[0]);
- Path path = new Path(s[1]+"/peer"+loc);
- FileSystem fs = path.getFileSystem(conf);
- SequenceFile.Writer writer
- = new SequenceFile.Writer(fs,conf,path,
- MRContainer.class,MRContainer.class);
- long count = 0;
- for ( MRData e: (Bag)(local_cache.get(out_num)) ) {
- key.set(count++);
- data_container.set(e);
- writer.append(key_container,data_container);
- };
- writer.close();
- };
- } else {
- // the BSP result is an aggregation:
- // send the partial results to the master peer
- peer.send(masterTask,new MRContainer(local_cache.get(source_num)));
- peer.sync();
- if (peer.getPeerName().equals(masterTask)) {
- // only the master peer collects the partial aggregations
- MRContainer msg;
- while ((msg = peer.getCurrentMessage()) != null)
- acc_result = acc_fnc.eval(new Tuple(acc_result,msg.data()));
- // write the final aggregation result
- peer.write(null_key,new MRContainer(acc_result));
- }
- }
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void setup ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer ) {
- try {
- super.setup(peer);
- Configuration conf = peer.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- all_peer_names = peer.getAllPeerNames();
- all_peers = new BSPPeerImpl[all_peer_names.length];
- Arrays.sort(all_peer_names); // is this necessary?
- source_num = conf.getInt("mrql.output.tag",0);
- Tree code = Tree.parse(conf.get("mrql.superstep"));
- superstep_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.initial.state"));
- state = Interpreter.evalE(code);
- if (conf.get("mrql.zero") != null && !conf.get("mrql.zero").equals("")) {
- code = Tree.parse(conf.get("mrql.zero"));
- acc_result = Interpreter.evalE(code);
- code = Tree.parse(conf.get("mrql.accumulator"));
- acc_fnc = functional_argument(conf,code);
- } else acc_result = null;
- orderp = conf.getBoolean("mrql.orderp",false);
- masterTask = all_peer_names[peer.getNumPeers()/2];
- msg_cache = new Bag(1000);
- local_cache = new Tuple(max_input_files);
- for ( int i = 0; i < max_input_files; i++ )
- local_cache.set(i,new Bag());
- } catch (Exception e) {
- e.printStackTrace();
- throw new Error("Cannot setup the Hama BSP job: "+e);
- }
- }
-
- @Override
- public void cleanup ( BSPPeer<MRContainer,MRContainer,MRContainer,MRContainer,MRContainer> peer ) throws IOException {
- if (!Config.local_mode)
- clean();
- local_cache = null;
- super.cleanup(peer);
- }
- }
-
- /** set Hama's min split size and number of BSP tasks (doesn't work with Hama 0.6.0) */
- public static void setupSplits ( BSPJob job, DataSet ds ) throws IOException {
- long[] sizes = new long[ds.source.size()];
- if (sizes.length > Config.nodes)
- throw new Error("Cannot distribute "+sizes.length+" files over "+Config.nodes+" BSP tasks");
- for ( int i = 0; i < sizes.length; i++ )
- sizes[i] = ds.source.get(i).size(Plan.conf);
- long total_size = 0;
- for ( long size: sizes )
- total_size += size;
- long split_size = Math.max(total_size/Config.nodes,100000);
- int tasks = 0;
- do { // adjust split_size
- tasks = 0;
- for ( long size: sizes )
- tasks += (int)Math.ceil(size/(double)split_size);
- if (tasks > Config.nodes)
- split_size = (long)Math.ceil((double)split_size*1.01);
- } while (tasks > Config.nodes);
- job.setNumBspTask(tasks);
- System.err.println("*** Using "+tasks+" BSP tasks (out of a max "+Config.nodes+")."
- +" Each task will handle about "+Math.min(total_size/Config.nodes,split_size)
- +" bytes of input data.");
- job.set("bsp.min.split.size",Long.toString(split_size));
- }
-
- /** Evaluate a BSP operation that returns a DataSet
- * @param source_nums output tags
- * @param superstep the superstep function
- * @param init_state initial state
- * @param orderp do we need to order the result?
- * @param source input dataset
- * @return a new data source that contains the result
- */
- public final static MRData BSP ( int[] source_nums, // output tags
- Tree superstep, // superstep function
- Tree init_state, // initial state
- boolean orderp, // do we need to order the result?
- DataSet source // input dataset
- ) throws Exception {
- String[] newpaths = new String[source_nums.length];
- newpaths[0] = new_path(conf);
- conf.set("mrql.output.paths",source_nums[0]+":"+newpaths[0]);
- for ( int i = 1; i < source_nums.length; i++ ) {
- newpaths[i] = new_path(conf);
- Path path = new Path(newpaths[1]);
- FileSystem fs = path.getFileSystem(conf);
- fs.mkdirs(path);
- conf.set("mrql.output.paths",conf.get("mrql.output.paths")+","+source_nums[i]+":"+newpaths[i]);
- };
- conf.set("mrql.superstep",superstep.toString());
- conf.set("mrql.initial.state",init_state.toString());
- conf.set("mrql.zero","");
- conf.setInt("mrql.output.tag",source_nums[0]);
- conf.setBoolean("mrql.orderp",orderp);
- BSPJob job = new BSPJob((HamaConfiguration)conf,BSPop.class);
- setupSplits(job,source);
- job.setJobName(newpaths[0]);
- distribute_compiled_arguments(getConfiguration(job));
- job.setBspClass(BSPop.class);
- Path outpath = new Path(newpaths[0]);
- job.setOutputPath(outpath);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setOutputFormat(SequenceFileOutputFormat.class);
- job.setInputFormat(MultipleBSPInput.class);
- FileInputFormat.setInputPaths(job,source.merge());
- job.waitForCompletion(true);
- if (source_nums.length == 1) {
- BinaryDataSource ds = new BinaryDataSource(source_nums[0],newpaths[0],conf);
- ds.to_be_merged = orderp;
- return new MR_dataset(new DataSet(ds,0,3));
- } else {
- MRData[] s = new MRData[source_nums.length];
- for ( int i = 0; i < source_nums.length; i++ ) {
- BinaryDataSource ds = new BinaryDataSource(source_nums[i],newpaths[i],conf);
- ds.to_be_merged = orderp;
- s[i] = new MR_dataset(new DataSet(ds,0,3));
- };
- return new Tuple(s);
- }
- }
-
- /** Evaluate a BSP operation that aggregates the results
- * @param source_num output tag
- * @param superstep the superstep function
- * @param init_state initial state
- * @param acc_fnc accumulator function
- * @param zero zero value for the accumulator
- * @param source input dataset
- * @return the aggregation result
- */
- public final static MRData BSPaggregate ( int source_num, // output tag
- Tree superstep, // superstep function
- Tree init_state, // initial state
- Tree acc_fnc, // accumulator function
- Tree zero, // zero value for the accumulator
- DataSet source // input dataset
- ) throws Exception {
- String newpath = new_path(conf);
- conf.set("mrql.superstep",superstep.toString());
- conf.set("mrql.initial.state",init_state.toString());
- conf.set("mrql.accumulator",acc_fnc.toString());
- conf.set("mrql.zero",zero.toString());
- conf.setInt("mrql.output.tag",source_num);
- conf.setBoolean("mrql.orderp",false);
- BSPJob job = new BSPJob((HamaConfiguration)conf,BSPop.class);
- setupSplits(job,source);
- job.setJobName(newpath);
- distribute_compiled_arguments(getConfiguration(job));
- job.setBspClass(BSPop.class);
- Path outpath = new Path(newpath);
- job.setOutputPath(outpath);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setOutputFormat(SequenceFileOutputFormat.class);
- job.setInputFormat(MultipleBSPInput.class);
- FileInputFormat.setInputPaths(job,source.merge());
- job.waitForCompletion(true);
- FileSystem fs = outpath.getFileSystem(conf);
- FileStatus[] files = fs.listStatus(outpath);
- for ( int i = 0; i < files.length; i++ ) {
- SequenceFile.Reader sreader = new SequenceFile.Reader(fs,files[i].getPath(),conf);
- MRContainer key = new MRContainer();
- MRContainer value = new MRContainer();
- sreader.next(key,value);
- sreader.close();
- if (value.data() != null)
- return value.data();
- };
- return null;
- }
- }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/MultipleBSPInput.java
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/MultipleBSPInput.java b/src/main/java/BSP/MultipleBSPInput.java
deleted file mode 100644
index aa43423..0000000
--- a/src/main/java/BSP/MultipleBSPInput.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.Trees;
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hama.bsp.*;
-
-
-/** A FileInputFormat for multiple files, where each file may be associated with
- * a different FileInputFormat */
-final public class MultipleBSPInput extends BSPMRQLFileInputFormat {
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split, BSPJob job ) throws IOException {
- if (Evaluator.evaluator == null)
- try {
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
- } catch (Exception ex) {
- throw new Error(ex);
- };
- String path = ((FileSplit)split).getPath().toString();
- Configuration conf = BSPPlan.getConfiguration(job);
- DataSource ds = DataSource.get(path,conf);
- Plan.conf = conf;
- if (ds instanceof ParsedDataSource)
- return new BSPParsedInputFormat.ParsedRecordReader((FileSplit)split,
- job,
- ((ParsedDataSource)ds).parser,
- ds.source_num,
- (Trees)((ParsedDataSource)ds).args);
- else if (ds instanceof BinaryDataSource)
- return new BSPBinaryInputFormat.BinaryInputRecordReader((FileSplit)split,job,ds.source_num);
- else if (ds instanceof GeneratorDataSource)
- return new BSPGeneratorInputFormat.GeneratorRecordReader((FileSplit)split,ds.source_num,job);
- else throw new Error("Unknown data source: "+ds+" for path "+path);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/CrossProductOperation.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/CrossProductOperation.java b/src/main/java/MapReduce/CrossProductOperation.java
deleted file mode 100644
index a9216cc..0000000
--- a/src/main/java/MapReduce/CrossProductOperation.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.net.URI;
-import java.util.List;
-import java.util.Vector;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.filecache.DistributedCache;
-
-
-/** The CrossProduct physical operation (similar to block-nested loop) */
-final public class CrossProductOperation extends MapReducePlan {
-
- /** The mapper for the CrossProduct operation */
- private final static class crossProductMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
- private static String counter; // a Hadoop user-defined counter used in the repeat operation
- private static Function reduce_fnc; // the reduce function
- private static Function map_fnc; // the mapper function
- private static DataSet cached_dataset;
- private final static List<MRData> outer
- = new Vector<MRData>(Config.map_cache_size); // fix-size cache for the outer
- private static int index;
- private static MRContainer last_key;
- private static URI[] uris;
- private static Path[] local_paths;
- private static Function acc_fnc; // aggregator
- private static MRData result; // aggregation result
- private static Tuple pair = new Tuple(2);
- private static MRContainer container = new MRContainer(new MR_int(0));
-
- private void write ( MRContainer key, MRData value, Context context )
- throws IOException, InterruptedException {
- if (result != null) { // aggregation
- pair.set(0,result);
- pair.set(1,value);
- result = acc_fnc.eval(pair);
- } else if (counter.equals("-")) {
- container.set(value);
- context.write(key,container);
- } else { // increment the repetition counter if the repeat condition is true
- Tuple t = (Tuple)value;
- if (((MR_bool)t.second()).get())
- context.getCounter("mrql",counter).increment(1);
- container.set(t.first());
- context.write(key,container);
- }
- }
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- try {
- last_key = key;
- for ( MRData x: (Bag)map_fnc.eval(value.data()) )
- if (index++ == Config.map_cache_size) {
- for ( MRData y: cached_data(context.getConfiguration()) ) {
- pair.set(1,y);
- for ( MRData z: outer ) {
- pair.set(0,z);
- for ( MRData v: (Bag)reduce_fnc.eval(pair) )
- write(key,v,context);
- }
- };
- index = 0;
- outer.clear();
- } else outer.add(x);
- } catch (Exception e) {
- throw new Error("Cannot perform the crossProduct: "+e);
- }
- }
-
- protected Bag cached_data ( final Configuration conf ) {
- try {
- Bag res = new Bag();
- final FileSystem fs = FileSystem.getLocal(conf);
- for ( int i = 0; i < local_paths.length; i++ ) {
- // hadoop 0.20.2 distributed cache doesn't work in stand-alone
- final Path path = (conf.get("mapred.job.tracker").equals("local"))
- ? new Path(uris[i].toString())
- : local_paths[i];
- if (path.getName().endsWith(".jar"))
- continue;
- res = res.union(new Bag(new BagIterator () {
- final SequenceFile.Reader reader = new SequenceFile.Reader(fs,path,conf);
- final MRContainer key = new MRContainer(new MR_int(0));
- final MRContainer value = new MRContainer(new MR_int(0));
- public boolean hasNext () {
- try {
- boolean done = reader.next(key,value);
- if (!done)
- reader.close();
- return done;
- } catch (IOException e) {
- throw new Error("Cannot collect values from distributed cache");
- }
- }
- public MRData next () {
- return value.data();
- }
- }));
- };
- return res;
- } catch (Exception e) {
- throw new Error("Cannot setup the cross product: "+e);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.reducer"));
- reduce_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.mapper"));
- map_fnc = functional_argument(conf,code);
- if (conf.get("mrql.zero") != null) {
- code = Tree.parse(conf.get("mrql.zero"));
- result = Interpreter.evalE(code);
- code = Tree.parse(conf.get("mrql.accumulator"));
- acc_fnc = functional_argument(conf,code);
- } else result = null;
- counter = conf.get("mrql.counter");
- uris = DistributedCache.getCacheFiles(conf);
- local_paths = DistributedCache.getLocalCacheFiles(conf);
- index = 0;
- } catch (Exception e) {
- throw new Error("Cannot setup the crossProduct: "+e);
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (index > 0)
- try {
- for ( MRData y: cached_data(context.getConfiguration()) ) {
- pair.set(1,y);
- for ( MRData z: outer ) {
- pair.set(0,z);
- for ( MRData v: (Bag)reduce_fnc.eval(pair) )
- write(last_key,v,context);
- }
- };
- } catch (Exception e) {
- throw new Error("Cannot cleanup the crossProduct: "+e);
- };
- index = 0;
- outer.clear();
- if (result != null) // emit the result of aggregation
- context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
- super.cleanup(context);
- }
- }
-
- /** The CrossProduct physical operator (similar to block-nested loop)
- * @param mx left mapper
- * @param my right mapper
- * @param reduce_fnc reducer
- * @param acc_fnc optional accumulator function
- * @param zero optional the zero value for the accumulator
- * @param X the left source
- * @param Y the right source (stored in distributed cache)
- * @param stop_counter optional counter used in repeat operation
- * @return a new data source that contains the result
- */
- public final static DataSet crossProduct ( Tree mx, // left mapper
- Tree my, // right mapper
- Tree reduce_fnc, // reducer
- Tree acc_fnc, // optional accumulator function
- Tree zero, // optional the zero value for the accumulator
- DataSet X, // the left source
- DataSet Y, // the right source (stored in distributed cache)
- String stop_counter ) // optional counter used in repeat operation
- throws Exception {
- DataSet ds = MapOperation.cMap(my,null,null,Y,"-");
- String newpath = new_path(conf);
- conf.set("mrql.reducer",reduce_fnc.toString());
- conf.set("mrql.mapper",mx.toString());
- if (zero != null) {
- conf.set("mrql.accumulator",acc_fnc.toString());
- conf.set("mrql.zero",zero.toString());
- conf.set("mapred.min.split.size","268435456");
- } else conf.set("mrql.zero","");
- conf.set("mrql.counter",stop_counter);
- Job job = new Job(conf,newpath);
- distribute_compiled_arguments(job.getConfiguration());
- job.setJarByClass(MapReducePlan.class);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setOutputFormatClass(SequenceFileOutputFormat.class);
- PathFilter pf = new PathFilter () { public boolean accept ( Path path ) {
- return !path.getName().startsWith("_");
- } };
- for (DataSource p: ds.source) {
- Path path = new Path(p.path);
- for ( FileStatus s: path.getFileSystem(conf).listStatus(path,pf) )
- DistributedCache.addCacheFile(s.getPath().toUri(),job.getConfiguration());
- };
- for (DataSource p: X.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,crossProductMapper.class);
- FileOutputFormat.setOutputPath(job,new Path(newpath));
- job.setNumReduceTasks(0);
- job.waitForCompletion(true);
- long c = (stop_counter.equals("-")) ? 0
- : job.getCounters().findCounter("mrql",stop_counter).getValue();
- return new DataSet(new BinaryDataSource(newpath,conf),c,outputRecords(job));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/GroupByJoinPlan.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/GroupByJoinPlan.java b/src/main/java/MapReduce/GroupByJoinPlan.java
deleted file mode 100644
index 4f93e1e..0000000
--- a/src/main/java/MapReduce/GroupByJoinPlan.java
+++ /dev/null
@@ -1,464 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-
-
-/**
- * A map-reduce job that captures a join with group-by. Similar to matrix multiplication.<br/>
- * It captures queries of the form:
- * <pre>
- * select r(kx,ky,c(z))
- * from x in X, y in Y, z = mp(x,y)
- * where jx(x) = jy(y)
- * group by (kx,ky): (gx(x),gy(y));
- * </pre>
- * where: mp: map function, r: reduce function, c: combine function,
- * jx: left join key function, jy: right join key function,
- * gx: left group-by function, gy: right group-by function.
- * <br/>
- * Example: matrix multiplication:
- * <pre>
- * select ( sum(z), i, j )
- * from (x,i,k) in X, (y,k,j) in Y, z = x*y
- * group by (i,j);
- * </pre>
- * It uses m*n partitions, so that n/m=|X|/|Y| and a hash table of size |X|/n*|Y|/m can fit in memory M.
- * That is, n = |X|/sqrt(M), m = |Y|/sqrt(M).
- * Each partition generates |X|/n*|Y|/m data. It replicates X n times and Y m times.
- * Uses a hash-table H of size |X|/n*|Y|/m
- * MapReduce pseudo-code:
- * <pre>
- * mapX ( x )
- * for i = 0,n-1
- * emit ( ((hash(gx(x)) % m)+m*i, jx(x), 1), (1,x) )
- *
- * mapY ( y )
- * for i = 0,m-1
- * emit ( ((hash(gy(y)) % n)*m+i, jy(y), 2), (2,y) )
- * </pre>
- * mapper output key: (partition,joinkey,tag), value: (tag,data) <br/>
- * Partitioner: over partition <br/>
- * GroupingComparator: over partition and joinkey <br/>
- * SortComparator: major partition, minor joinkey, sub-minor tag <br/>
- * <pre>
- * reduce ( (p,_,_), s )
- * if p != current_partition
- * flush()
- * current_partition = p
- * read x from s first and store it to xs
- * for each y from the rest of s
- * for each x in xs
- * H[(gx(x),gy(y))] = c( H[(gx(x),gy(y))], mp((x,y)) )
- * </pre>
- * where flush() is: for each ((kx,ky),v) in H: emit r((kx,ky),v)
- */
-final public class GroupByJoinPlan extends Plan {
-
- /** mapper output key: (partition,joinkey,tag) */
- private final static class GroupByJoinKey implements Writable {
- public int partition; // one of n*m
- public byte tag; // 1 or 2
- public MRData key;
-
- GroupByJoinKey () {}
- GroupByJoinKey ( int p, byte t, MRData k ) {
- partition = p;
- tag = t;
- key = k;
- }
-
- public void write ( DataOutput out ) throws IOException {
- out.writeByte(tag);
- WritableUtils.writeVInt(out,partition);
- key.write(out);
- }
-
- public void readFields ( DataInput in ) throws IOException {
- tag = in.readByte();
- partition = WritableUtils.readVInt(in);
- key = MRContainer.read(in);
- }
-
- public String toString () { return "["+partition+","+tag+","+key+"]"; }
- }
-
- /** partition based on key.partition only */
- private final static class GroupByJoinPartitioner extends Partitioner<GroupByJoinKey,MRContainer> {
- final public int getPartition ( GroupByJoinKey key, MRContainer value, int numPartitions ) {
- return key.partition % numPartitions;
- }
- }
-
- /** sorting with major order key.partition, minor key.key, minor key.tag */
- private final static class GroupByJoinSortComparator implements RawComparator<GroupByJoinKey> {
- int[] container_size;
-
- public GroupByJoinSortComparator () {
- container_size = new int[1];
- }
-
- final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
- try {
- int c = WritableComparator.readVInt(x,xs+1)-WritableComparator.readVInt(y,ys+1);
- if (c != 0)
- return c;
- int tsize = 1+WritableUtils.decodeVIntSize(x[xs+1]);
- c = MRContainer.compare(x,xs+tsize,xl-tsize,y,ys+tsize,yl-tsize,container_size);
- if (c != 0)
- return c;
- return x[xs] - y[ys];
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- final public int compare ( GroupByJoinKey x, GroupByJoinKey y ) {
- int c = x.partition - y.partition;
- if (c != 0)
- return c;
- c = x.key.compareTo(y.key);
- if (c != 0)
- return c;
- return x.tag - y.tag;
- }
- }
-
- /** grouping by key.partition and key.key */
- private final static class GroupByJoinGroupingComparator implements RawComparator<GroupByJoinKey> {
- int[] container_size;
-
- public GroupByJoinGroupingComparator() {
- container_size = new int[1];
- }
-
- final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
- try {
- int c = WritableComparator.readVInt(x,xs+1)-WritableComparator.readVInt(y,ys+1);
- if (c != 0)
- return c;
- int tsize = 1+WritableUtils.decodeVIntSize(x[xs+1]);
- return MRContainer.compare(x,xs+tsize,xl-tsize,y,ys+tsize,yl-tsize,container_size);
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- final public int compare ( GroupByJoinKey x, GroupByJoinKey y ) {
- int c = x.partition - y.partition;
- return (c != 0) ? c : x.key.compareTo(y.key);
- }
- }
-
- /** the left GroupByJoin mapper */
- private final static class MapperLeft extends Mapper<MRContainer,MRContainer,GroupByJoinKey,MRContainer> {
- private static int n, m;
- private static Function left_join_key_fnc;
- private static Function left_groupby_fnc;
- private static GroupByJoinKey ckey = new GroupByJoinKey(0,(byte)1,new MR_int(0));
- private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(1));
- private static MRContainer cvalue = new MRContainer(tvalue);
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- MRData data = value.data();
- for ( int i = 0; i < n; i++ ) {
- ckey.partition = (left_groupby_fnc.eval(data).hashCode() % m)+m*i;
- ckey.key = left_join_key_fnc.eval(data);
- tvalue.set(1,data);
- context.write(ckey,cvalue);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.join.key.left"));
- left_join_key_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.groupby.left"));
- left_groupby_fnc = functional_argument(conf,code);
- m = conf.getInt("mrql.m",1);
- n = conf.getInt("mrql.n",1);
- } catch (Exception e) {
- throw new Error("Cannot retrieve the left mapper plan");
- }
- }
- }
-
- /** the right GroupByJoin mapper */
- private final static class MapperRight extends Mapper<MRContainer,MRContainer,GroupByJoinKey,MRContainer> {
- private static int n, m;
- private static Function right_join_key_fnc;
- private static Function right_groupby_fnc;
- private static GroupByJoinKey ckey = new GroupByJoinKey(0,(byte)2,new MR_int(0));
- private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(2));
- private static MRContainer cvalue = new MRContainer(tvalue);
-
- @Override
- public void map ( MRContainer key, MRContainer value, Context context )
- throws IOException, InterruptedException {
- MRData data = value.data();
- for ( int i = 0; i < m; i++ ) {
- ckey.partition = (right_groupby_fnc.eval(data).hashCode() % n)*m+i;
- ckey.key = right_join_key_fnc.eval(data);
- tvalue.set(1,data);
- context.write(ckey,cvalue);
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.join.key.right"));
- right_join_key_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.groupby.right"));
- right_groupby_fnc = functional_argument(conf,code);
- m = conf.getInt("mrql.m",1);
- n = conf.getInt("mrql.n",1);
- } catch (Exception e) {
- throw new Error("Cannot retrieve the right mapper plan");
- }
- }
- }
-
- /** the GroupByJoin reducer */
- private static class JoinReducer extends Reducer<GroupByJoinKey,MRContainer,MRContainer,MRContainer> {
- private static String counter; // a Hadoop user-defined counter used in the repeat operation
- private static int n, m; // n*m partitioners
- private static Function left_groupby_fnc; // left group-by function
- private static Function right_groupby_fnc;// right group-by function
- private static Function map_fnc; // the map function
- private static Function combine_fnc; // the combine function
- private static Function reduce_fnc; // the reduce function
- private static Bag left = new Bag(); // a cached bag of input fragments from left input
- private static int current_partition = -1;
- private static Hashtable<MRData,MRData> hashTable; // in-reducer combiner
- private static Tuple pair = new Tuple(2);
- private static MRContainer ckey = new MRContainer(new MR_int(0));
- private static MRContainer cvalue = new MRContainer(new MR_int(0));
- private static MRContainer container = new MRContainer(new MR_int(0));
- private static Tuple tkey = new Tuple(2);
- private static Bag tbag = new Bag(2);
-
- private static void write ( MRContainer key, MRData value, Context context )
- throws IOException, InterruptedException {
- if (counter.equals("-")) {
- container.set(value);
- context.write(key,container);
- } else { // increment the repetition counter if the repeat condition is true
- Tuple t = (Tuple)value;
- if (((MR_bool)t.second()).get())
- context.getCounter("mrql",counter).increment(1);
- container.set(t.first());
- context.write(key,container);
- }
- }
-
- private void store ( MRData key, MRData value ) throws IOException {
- MRData old = hashTable.get(key);
- Tuple k = (Tuple)key;
- pair.set(0,key);
- for ( MRData e: (Bag)map_fnc.eval(value) )
- if (old == null)
- hashTable.put(key,e);
- else {
- tbag.clear();
- tbag.add_element(e).add_element(old);
- pair.set(1,tbag);
- for ( MRData z: (Bag)combine_fnc.eval(pair) )
- hashTable.put(key,z); // normally, done once
- }
- }
-
- protected static void flush_table ( Context context ) throws IOException, InterruptedException {
- Enumeration<MRData> en = hashTable.keys();
- while (en.hasMoreElements()) {
- MRData key = en.nextElement();
- MRData value = hashTable.get(key);
- ckey.set(key);
- pair.set(0,key);
- tbag.clear();
- tbag.add_element(value);
- pair.set(1,tbag);
- for ( MRData e: (Bag)reduce_fnc.eval(pair) )
- write(ckey,e,context);
- };
- hashTable.clear();
- }
-
- @Override
- public void reduce ( GroupByJoinKey key, Iterable<MRContainer> values, Context context )
- throws IOException, InterruptedException {
- if (key.partition != current_partition && current_partition > 0) {
- // at the end of a partition, flush the hash table
- flush_table(context);
- current_partition = key.partition;
- };
- left.clear();
- Tuple p = null;
- final Iterator<MRContainer> i = values.iterator();
- // left tuples arrive before right tuples; cache the left values into the left bag
- while (i.hasNext()) {
- p = (Tuple)i.next().data();
- if (((MR_byte)p.first()).get() == 2)
- break;
- left.add(p.second());
- p = null;
- };
- // the previous value was from right
- if (p != null) {
- MRData y = p.second();
- MRData gy = right_groupby_fnc.eval(y);
- // cross product with left (must use new Tuples)
- for ( MRData x: left )
- store(new Tuple(left_groupby_fnc.eval(x),gy),new Tuple(x,y));
- // the rest of values are from right
- while (i.hasNext()) {
- y = ((Tuple)i.next().data()).second();
- gy = right_groupby_fnc.eval(y);
- // cross product with left (must use new Tuples)
- for ( MRData x: left )
- store(new Tuple(left_groupby_fnc.eval(x),gy),new Tuple(x,y));
- }
- }
- }
-
- @Override
- protected void setup ( Context context ) throws IOException,InterruptedException {
- super.setup(context);
- try {
- Configuration conf = context.getConfiguration();
- Config.read(conf);
- if (Plan.conf == null)
- Plan.conf = conf;
- Tree code = Tree.parse(conf.get("mrql.groupby.left"));
- left_groupby_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.groupby.right"));
- right_groupby_fnc = functional_argument(conf,code);
- m = conf.getInt("mrql.m",1);
- n = conf.getInt("mrql.n",1);
- code = Tree.parse(conf.get("mrql.mapper"));
- map_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.combiner"));
- combine_fnc = functional_argument(conf,code);
- code = Tree.parse(conf.get("mrql.reducer"));
- reduce_fnc = functional_argument(conf,code);
- counter = conf.get("mrql.counter");
- hashTable = new Hashtable<MRData,MRData>(Config.map_cache_size);
- } catch (Exception e) {
- throw new Error("Cannot retrieve the reducer plan");
- }
- }
-
- @Override
- protected void cleanup ( Context context ) throws IOException,InterruptedException {
- if (hashTable != null)
- flush_table(context);
- hashTable = null; // garbage-collect it
- super.cleanup(context);
- }
- }
-
- /** the GroupByJoin operation
- * @param left_join_key_fnc left join key function
- * @param right_join_key_fnc right join key function
- * @param left_groupby_fnc left group-by function
- * @param right_groupby_fnc right group-by function
- * @param map_fnc map function
- * @param combine_fnc combine function
- * @param reduce_fnc reduce function
- * @param X left data set
- * @param Y right data set
- * @param num_reducers number of reducers
- * @param n left dimension of the reducer grid
- * @param m right dimension of the reducer grid
- * @param stop_counter optional counter used in repeat operation
- * @return a DataSet that contains the result
- */
- public final static DataSet groupByJoin
- ( Tree left_join_key_fnc, // left join key function
- Tree right_join_key_fnc, // right join key function
- Tree left_groupby_fnc, // left group-by function
- Tree right_groupby_fnc, // right group-by function
- Tree map_fnc, // map function
- Tree combine_fnc, // combine function
- Tree reduce_fnc, // reduce function
- DataSet X, // left data set
- DataSet Y, // right data set
- int num_reducers, // number of reducers
- int n, int m, // dimensions of the reducer grid
- String stop_counter ) // optional counter used in repeat operation
- throws Exception {
- String newpath = new_path(conf);
- conf.set("mrql.join.key.left",left_join_key_fnc.toString());
- conf.set("mrql.join.key.right",right_join_key_fnc.toString());
- conf.set("mrql.groupby.left",left_groupby_fnc.toString());
- conf.set("mrql.groupby.right",right_groupby_fnc.toString());
- conf.setInt("mrql.m",m);
- conf.setInt("mrql.n",n);
- conf.set("mrql.mapper",map_fnc.toString());
- conf.set("mrql.combiner",combine_fnc.toString());
- conf.set("mrql.reducer",reduce_fnc.toString());
- conf.set("mrql.counter",stop_counter);
- Job job = new Job(conf,newpath);
- distribute_compiled_arguments(job.getConfiguration());
- job.setMapOutputKeyClass(GroupByJoinKey.class);
- job.setJarByClass(GroupByJoinPlan.class);
- job.setOutputKeyClass(MRContainer.class);
- job.setOutputValueClass(MRContainer.class);
- job.setPartitionerClass(GroupByJoinPartitioner.class);
- job.setSortComparatorClass(GroupByJoinSortComparator.class);
- job.setGroupingComparatorClass(GroupByJoinGroupingComparator.class);
- job.setOutputFormatClass(SequenceFileOutputFormat.class);
- FileOutputFormat.setOutputPath(job,new Path(newpath));
- for (DataSource p: X.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperLeft.class);
- for (DataSource p: Y.source)
- MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperRight.class);
- job.setReducerClass(JoinReducer.class);
- if (num_reducers > 0)
- job.setNumReduceTasks(num_reducers);
- job.waitForCompletion(true);
- long c = (stop_counter.equals("-")) ? 0
- : job.getCounters().findCounter("mrql",stop_counter).getValue();
- DataSource s = new BinaryDataSource(newpath,conf);
- s.to_be_merged = false;
- return new DataSet(s,c,MapReducePlan.outputRecords(job));
- }
-}
[02/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/SymbolTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/SymbolTable.java b/src/main/java/gen/SymbolTable.java
deleted file mode 100644
index 52ea868..0000000
--- a/src/main/java/gen/SymbolTable.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-import java.util.Iterator;
-
-
-/* A symbol in the symbol table */
-class SymbolCell {
- String name;
- Tree binding;
- SymbolCell next;
- SymbolCell ( String n, Tree v, SymbolCell r ) { name=n; binding=v; next=r; }
-}
-
-
-final class SymbolTableIterator implements Iterator<String> {
- SymbolTable table;
- SymbolTable original_table;
-
- SymbolTableIterator ( SymbolTable t ) {
- table = new SymbolTable();
- original_table = t;
- for ( int i = 0; i < t.symbol_table_size; i++ )
- table.symbol_table[i] = t.symbol_table[i];
- for ( int i = 0; i < t.scope_stack.length; i++ )
- table.scope_stack[i] = t.scope_stack[i];
- table.scope_stack_top = t.scope_stack_top;
- }
-
- public boolean hasNext () {
- if (table.scope_stack_top <= 0)
- return false;
- int n = table.scope_stack[table.scope_stack_top-1];
- if (n < 0 || !table.symbol_table[n].binding.equals(original_table.lookup(table.symbol_table[n].name))) {
- table.scope_stack_top--;
- return hasNext();
- };
- return true;
- }
-
- public String next () {
- int n = table.scope_stack[--table.scope_stack_top];
- String var = table.symbol_table[n].name;
- table.symbol_table[n] = table.symbol_table[n].next;
- return var;
- }
-
- public void remove () {}
-}
-
-
-public class SymbolTable implements Iterable<String> {
- final static int symbol_table_size = 997;
- final static int initial_scope_stack_length = 1000;
-
- SymbolCell[] symbol_table;
- int[] scope_stack;
- int scope_stack_top = 0;
-
- public SymbolTable () {
- symbol_table = new SymbolCell[symbol_table_size];
- scope_stack = new int[initial_scope_stack_length];
- scope_stack_top = 0;
- for (int i = 0; i < symbol_table_size; i++)
- symbol_table[i] = null;
- }
-
- public Iterator<String> iterator () {
- return new SymbolTableIterator(this);
- }
-
- /* a hashing function for strings */
- int hash ( String s ) {
- return Math.abs(s.hashCode()) % symbol_table_size;
- }
-
- /* insert a new item in the symbol table */
- public void insert ( String key, Tree binding ) {
- int loc = hash(key);
- symbol_table[loc] = new SymbolCell(key,binding,symbol_table[loc]);
- if (scope_stack_top >= scope_stack.length) {
- int[] v = new int[scope_stack.length*2];
- for ( int i = 0; i < scope_stack.length; i++ )
- v[i] = scope_stack[i];
- scope_stack = v;
- };
- scope_stack[scope_stack_top++] = loc;
- }
-
- /* replace an item with a given name in the symbol table */
- public void replace ( String key, Tree binding ) {
- int loc = hash(key);
- for (SymbolCell s = symbol_table[loc]; s != null; s=s.next)
- if (s.name.equals(key))
- s.binding = binding;
- }
-
- /* remove an item with a given name from the symbol table */
- public boolean remove ( String key ) {
- int loc = hash(key);
- SymbolCell prev = symbol_table[loc];
- if (prev == null)
- return false;
- if (prev.name.equals(key)) {
- symbol_table[loc] = prev.next;
- return true;
- };
- for (SymbolCell s = prev.next; s != null; s=s.next, prev=prev.next)
- if (s.name.equals(key)) {
- prev.next = s.next;
- return true;
- };
- return false;
- }
-
- /* lookup for an item in the symbol table */
- public Tree lookup ( String key ) {
- int loc = hash(key);
- for (SymbolCell s = symbol_table[loc]; s != null; s=s.next)
- if (s.name.equals(key))
- return s.binding;
- return null; // if not found
- }
-
- /* return true if key is local */
- public boolean is_local ( String key ) {
- int loc = hash(key);
- int i = 0;
- for ( SymbolCell s = symbol_table[loc]; s != null; s = s.next, i++ )
- if (s.name.equals(key)) {
- int k = 0;
- for ( int j = scope_stack_top-1; j >= 0 && scope_stack[j] >= 0; j--)
- if (scope_stack[j] == loc)
- if (k++ == i)
- return true;
- return false;
- };
- return false; // if not found
- }
-
- /* start a new environment */
- public void begin_scope () {
- if (scope_stack_top >= scope_stack.length) {
- int[] v = new int[scope_stack.length*2];
- for ( int i = 0; i < scope_stack.length; i++ )
- v[i] = scope_stack[i];
- scope_stack = v;
- };
- scope_stack[scope_stack_top++] = -1;
- }
-
- /* pop the last environment */
- public void end_scope () {
- int i = scope_stack_top-1;
- for (; scope_stack[i]>=0 && i>0; i--) {
- int loc = scope_stack[i];
- symbol_table[loc] = symbol_table[loc].next;
- };
- scope_stack_top = i;
- }
-
- /* display the content of the symbol table */
- public void display () {
- SymbolCell[] s = new SymbolCell[symbol_table_size];
- for (int i = 0; i<symbol_table_size; i++)
- s[i] = symbol_table[i];
- for (int i = scope_stack_top-1; i>=0; i--)
- if (scope_stack[i] == -1)
- System.out.println("----------------");
- else {
- SymbolCell c = s[scope_stack[i]];
- s[scope_stack[i]] = c.next;
- System.out.println(c.name + ": " + c.binding);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Tree.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Tree.java b/src/main/java/gen/Tree.java
deleted file mode 100644
index 7ddc582..0000000
--- a/src/main/java/gen/Tree.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java_cup.runtime.*;
-import java.io.*;
-import java.util.HashMap;
-
-
-abstract public class Tree implements Serializable {
-
- /* the current line number in the parsed file */
- public static int line_number = 0;
-
- /* the current char position in the parsed file */
- public static int position_number = 0;
-
- /* true when Trees are parsed rather than processed */
- public static boolean parsed = false;
-
- /* the line number of the Tree in the parsed file */
- public int line;
-
- /* the char position of the Tree in the parsed file */
- public int position;
-
- Tree () { line = line_number; position = position_number; }
-
- /* deep equality */
- public abstract boolean equals ( Tree e );
-
- final public boolean is_node () { return (this instanceof Node); }
- final public boolean is_variable () { return (this instanceof VariableLeaf); }
- final public boolean is_long () { return (this instanceof LongLeaf); }
- final public boolean is_string () { return (this instanceof StringLeaf); }
- final public boolean is_double () { return (this instanceof DoubleLeaf); }
-
- final public String variableValue () { return (this instanceof VariableLeaf) ? ((VariableLeaf)this).value() : ""; }
- final public long longValue () { return (this instanceof LongLeaf) ? ((LongLeaf)this).value() : (long)0; }
- final public String stringValue () { return (this instanceof StringLeaf) ? ((StringLeaf)this).value() : ""; }
- final public double doubleValue () { return (this instanceof DoubleLeaf) ? ((DoubleLeaf)this).value() : (double)0.0; }
-
- /* size used for pretty() */
- protected abstract int size ();
-
- /* print the Tree into a string */
- public abstract String toString ();
-
- /* pretty-print the Tree padded with position space characters */
- public abstract String pretty ( int position );
-
- private static Tree fix_tree ( Tree e ) {
- if (e instanceof Node) {
- Trees cs = Trees.nil;
- for ( Tree a: ((Node) e).children().tail() )
- cs = cs.append(fix_tree(a));
- return new Node(((VariableLeaf)(((Node) e).children().head())).value(),cs);
- } else return e;
- }
-
- /* the inverse of toString() */
- final public static synchronized Tree parse ( String s ) throws Exception {
- GenParser.scanner = new GenLex(new StringReader("#<"+s+">"));
- GenParser.out = new PrintStream(new ByteArrayOutputStream());
- new GenParser(GenParser.scanner).parse();
- return fix_tree(GenParser.parse_tree);
- }
-
- private static HashMap<String,String> names = new HashMap<String,String>(1000);
-
- public static String add ( String s ) {
- String ns = names.get(s);
- if (ns == null) {
- names.put(s,s);
- return s;
- } else return ns;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Trees.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Trees.java b/src/main/java/gen/Trees.java
deleted file mode 100644
index 86f65ca..0000000
--- a/src/main/java/gen/Trees.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.util.Iterator;
-import java.io.*;
-
-
-final class TreeIterator implements Iterator<Tree> {
- Trees trees;
-
- TreeIterator ( Trees trees ) { this.trees = trees; }
-
- public boolean hasNext () { return trees.tail != null; }
-
- public Tree next () {
- Tree res = trees.head;
- trees = trees.tail;
- return res;
- }
-
- public void remove () { trees = trees.tail; }
-}
-
-
-final public class Trees implements Iterable<Tree>, Serializable {
- private final static int screen_size = 100;
- public Tree head;
- public Trees tail;
-
- public Trees ( Tree head, Trees tail ) {
- if (tail == null)
- throw new Error("Gen: an empty list of nodes must be nil, not null");
- this.head = head;
- this.tail = tail;
- }
-
- public Trees () {
- head = null;
- tail = null;
- }
-
- public final static Trees nil = new Trees();
-
- public Trees ( Tree head ) {
- this.head = head;
- tail = nil;
- }
-
- public Tree head () {
- if (tail == null)
- throw new Error("Gen: tried to retrieve the head of an empty list of nodes");
- return head;
- }
-
- public Trees tail () {
- if (tail == null)
- throw new Error("Gen: tried to retrieve the tail of an empty list of nodes");
- return tail;
- }
-
- public boolean is_empty () {
- return (tail == null);
- }
-
- /* number of nodes */
- public int length () {
- int n = 0;
- for (Trees r = this; !r.is_empty(); r = r.tail)
- n += 1;
- return n;
- }
-
- /* put an Tree e at the beginning of the nodes */
- public Trees cons ( Tree e ) {
- return new Trees(e,this);
- }
-
- /* put an Tree e at the end of the nodes */
- public Trees append ( Tree e ) {
- if (is_empty())
- return new Trees(e);
- else {
- Trees temp = new Trees(e,new Trees(e));
- Trees res = temp;
- for (Trees r = this; !r.is_empty(); r = r.tail) {
- temp.tail = temp.tail.cons(r.head);
- temp = temp.tail;
- };
- return res.tail;
- }
- }
-
- /* append two lists of nodes */
- public Trees append ( Trees s ) {
- if (is_empty())
- return s;
- else if (s.is_empty())
- return this;
- else {
- Trees temp = s.cons(s.head);
- Trees res = temp;
- for (Trees r = this; !r.is_empty(); r = r.tail)
- { temp.tail = temp.tail.cons(r.head);
- temp = temp.tail;
- }
- return res.tail;
- }
- }
-
- /* reverse the order of nodes */
- public Trees reverse () {
- Trees res = nil;
- for (Trees r = this; !r.is_empty(); r = r.tail)
- res = res.cons(r.head);
- return res;
- }
-
- /* is e one of the nodes? */
- public boolean member ( Tree e ) {
- for (Trees r = this; !r.is_empty(); r = r.tail)
- if (r.head.equals(e))
- return true;
- return false;
- }
-
- /* return the nth node */
- public Tree nth ( int n ) {
- Trees r = this;
- for (int i = 0; !r.is_empty() && i < n; r = r.tail(), i++)
- ;
- if (r.is_empty())
- throw new Error("Gen: tried to retrieve a nonexistent nth element from a list of nodes");
- else return r.head;
- }
-
- /* deep equality */
- public boolean equals ( Trees s ) {
- Trees n = this;
- Trees m = s;
- for(; n.tail != null && m.tail != null; n = n.tail, m = m.tail )
- if (!n.head.equals(m.head))
- return false;
- return (m.tail == null) && (n.tail == null);
- }
-
- protected int size () {
- int n = 1;
- for (Trees r = this; !r.is_empty(); r = r.tail)
- n += r.head.size()+1;
- return n;
- }
-
- public Iterator<Tree> iterator () { return new TreeIterator(this); }
-
- /* print the nodes */
- public String toString () {
- if (is_empty())
- return "()";
- String s = "(" + head;
- for (Trees r = tail; !r.is_empty(); r = r.tail)
- s = s + "," + r.head;
- return s + ")";
- }
-
- /* pretty-print the nodes */
- public String pretty ( int position ) {
- if (is_empty() || (position+size() <= screen_size))
- return toString();
- String s = "(" + head.pretty(position+1);
- for (Trees r=tail; !r.is_empty(); r=r.tail) {
- s = s + ",\n";
- for (int i=0; i<position+1; i++)
- s = s + " ";
- s = s + r.head.pretty(position+1);
- };
- return s + ")";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/VariableLeaf.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/VariableLeaf.java b/src/main/java/gen/VariableLeaf.java
deleted file mode 100644
index c4c0ed9..0000000
--- a/src/main/java/gen/VariableLeaf.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-
-
-final public class VariableLeaf extends Tree {
- public String value;
-
- public VariableLeaf ( String s ) {
- super();
- value = Tree.add(s);
- }
-
- final public String value () { return value; }
-
- public boolean equals ( Tree e ) {
- return (e instanceof VariableLeaf)
- && value == ((VariableLeaf) e).value;
- }
-
- protected int size () { return value.length(); }
-
- public String toString () { return value; }
-
- public String pretty ( int position ) { return value; }
-
- private void writeObject(ObjectOutputStream out) throws IOException {
- out.defaultWriteObject();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- value = Tree.add(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/gen.cup
----------------------------------------------------------------------
diff --git a/src/main/java/gen/gen.cup b/src/main/java/gen/gen.cup
deleted file mode 100644
index 91432ba..0000000
--- a/src/main/java/gen/gen.cup
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java_cup.runtime.*;
-import java.util.Stack;
-
-parser code {:
-
- static int[] tokens = {
- GenSym.error, GenSym.CHAR, GenSym.ID, GenSym.CSTRING, GenSym.LONG, GenSym.DOUBLE,
- GenSym.META, GenSym.MATCH, GenSym.CASE, GenSym.IS, GenSym.COLON, GenSym.COMMA,
- GenSym.GT, GenSym.LP, GenSym.RP, GenSym.LB, GenSym.RB, GenSym.LSB, GenSym.RSB,
- GenSym.BQ, GenSym.BQP, GenSym.DOTSP, GenSym.ANY, GenSym.OPER, GenSym.FAIL
- };
-
- static String[] token_names = {
- "error", "character", "identifier", "string", "integer", "float",
- "#<", "match", "case", "is", ":", ",", ">", "(", ")", "[", "]", "{", "}",
- "`", "`(", "...(", "_", "operator", "fail"
- };
-
- public static String print ( Symbol s ) {
- for (int i=0; i<tokens.length; i++)
- if (tokens[i]==s.sym)
- { String res = token_names[i] + " ";
- if (s.value==null)
- return res;
- else if (s.value instanceof Long)
- return res + ((Long) s.value).longValue();
- else if (s.value instanceof Double)
- return res + ((Double) s.value).doubleValue();
- else if (s.value instanceof String)
- return res + (String) s.value;
- }
- return "?";
- }
-
- public static java.io.PrintStream out;
-
- public static GenLex scanner;
-
- public static Tree parse_tree;
-
- public static int newlines = 0;
-
- public static int lcase = 0;
-
- public static Stack<String> labels = new Stack<String>();
-
- public void syntax_error ( Symbol token ) {
- throw new Error("*** Syntax Error: " + print(token) + " (line: " + scanner.line_pos()
- + ", position: " + scanner.char_pos() + ")");
- }
-
-:};
-
-/* Terminals (tokens returned by the scanner). */
-terminal META, METAL, MATCH, CASE, FAIL, IS, DOTS, COLON, COMMA, GT, LP, RP, LB, RB, LSB, RSB, BQ, BQP, DOTSP, ANY;
-
-terminal String CHAR;
-terminal String ID;
-terminal Long LONG;
-terminal Double DOUBLE;
-terminal String CSTRING;
-terminal String OPER;
-
-non terminal Tree exp, name, variable, rest;
-non terminal Trees expl;
-non terminal String case, cases, schar, jcode, scode;
-non terminal pcode, prog;
-
-precedence left OPER;
-precedence nonassoc META;
-precedence nonassoc MATCH;
-precedence nonassoc CASE;
-precedence nonassoc CSTRING;
-precedence nonassoc CHAR;
-
-start with prog;
-
-prog ::= pcode
- ;
-pcode ::= scode:s {: parser.out.print(s); :}
- | pcode scode:s {: parser.out.print(s); :}
- ;
-jcode ::= scode:s {: RESULT = s; :}
- | jcode:c scode:s {: RESULT = c+s; :}
- ;
-scode ::= schar:s {: RESULT = s; :}
- | LSB RSB {: RESULT = "{}"; :}
- | LSB jcode:c RSB {: RESULT = "{"+c+"}"; :}
- | METAL RB {: RESULT = "Trees.nil"; :}
- | META exp:e GT {: GenParser.parse_tree = e;
- RESULT = Meta.reify(e);
- for (int i=0; i < parser.newlines; i++)
- RESULT += "\n";
- parser.newlines = 0; :}
- | METAL expl:r RB {: RESULT = Meta.reify(new Node("Node",
- new Trees(new VariableLeaf("F"),r)));
- RESULT = RESULT.substring(13,RESULT.length()-1);
- for (int i=0; i < parser.newlines; i++)
- RESULT += "\n";
- parser.newlines = 0; :}
- | MATCH {: Meta.level++;
- GenLex.context.new_context();
- GenLex.context.in_match_body();
- :}
- jcode:c LSB cases:el RSB {: RESULT = "{ boolean FOUND_" + Meta.level + " = false; Tree E_"
- + Meta.level + " = " + c + "; " + el + " }";
- GenLex.context.close_context(GenParser.scanner);
- for (int i=0; i < parser.newlines; i++)
- RESULT += "\n";
- parser.newlines = 0;
- Meta.level--; :}
- | FAIL {: RESULT = "{ FOUND_" + Meta.level + "=false; break " + GenParser.labels.peek() + "; }"; :}
- ;
-schar ::= CHAR:c {: RESULT = c; :}
- | CSTRING:s {: RESULT = Crypt.encrypt(s); :}
- ;
-cases ::= jcode:j case:c {: RESULT = j+c; :}
- | cases:cl case:c {: RESULT = cl + " if (!FOUND_" + Meta.level + ") " + c; :}
- ;
-case ::= CASE {: GenParser.labels.push("LCASE_"+GenParser.lcase++); :}
- exp:e COLON jcode:j {: Condition m = Meta.pattern(e,"E_" + Meta.level);
- RESULT = "{ " + GenParser.labels.pop() + ": ";
- RESULT += (m.pred.equals("true")) ? "" : "if (" + m.pred + ") ";
- for (int i=0; i < parser.newlines; i++)
- RESULT += "\n";
- parser.newlines = 0;
- RESULT += "{ " + m.stmt + " FOUND_" + Meta.level
- + "=true; " + j + " }";
- for (int i = 0; i <= m.unmatched_brackets; i++)
- RESULT += "}";
- RESULT += ";"; :}
- ;
-name ::= ID:s {: RESULT = new VariableLeaf(s); :}
- ;
-variable ::= name:s {: RESULT = s; :}
- | BQ name:s {: RESULT = new Node("Escape",new Trees(s)); :}
- | BQP jcode:c RP {: RESULT = new Node("Code",new Trees(new StringLeaf(c))); :}
- | ANY {: RESULT = new VariableLeaf("_any_"); :}
- ;
-exp ::= LONG:n {: RESULT = new LongLeaf(n.longValue()); :}
- | DOUBLE:n {: RESULT = new DoubleLeaf(n.doubleValue()); :}
- | CSTRING:s {: RESULT = new StringLeaf(s); :}
- | variable:e {: RESULT = e; :}
- | variable:e LP RP {: RESULT = new Node("Node",new Trees(e)); :}
- | variable:e LP expl:el RP {: RESULT = new Node("Node",new Trees(e,el)); :}
- | BQ name:s LB exp:e RB {: RESULT = new Node("Higher",new Trees(s,new Trees(e))); :}
- | name:s IS exp:e {: RESULT = new Node("IS",new Trees(s,new Trees(e))); :}
- | LP exp:e RP {: RESULT = e; :}
- | exp:e1 OPER:o exp:e2 {: RESULT = new Node("Node",new Trees(new VariableLeaf(o),
- new Trees(e1,new Trees(e2)))); :}
- ;
-expl ::= exp:e {: RESULT = new Trees(e); :}
- | rest:e {: RESULT = new Trees(e); :}
- | expl:el COMMA rest:e {: RESULT = el.append(e); :}
- | expl:el COMMA exp:e {: RESULT = el.append(e); :}
- ;
-rest ::= DOTS name:n {: RESULT = new Node("Dots",new Trees(n)); :}
- | DOTSP jcode:c RP {: RESULT = new Node("Dots",new Trees(new StringLeaf(c))); :}
- | DOTS {: RESULT = new Node("Dots"); :}
- ;
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/gen.lex
----------------------------------------------------------------------
diff --git a/src/main/java/gen/gen.lex b/src/main/java/gen/gen.lex
deleted file mode 100644
index 06b073d..0000000
--- a/src/main/java/gen/gen.lex
+++ /dev/null
@@ -1,210 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java_cup.runtime.Symbol;
-
-final class Context {
- public int[] parentheses;
- public boolean[] match_begin;
- public int current;
- public Context () {
- current = 0;
- parentheses = new int[1000];
- match_begin = new boolean[1000];
- parentheses[0] = 0;
- match_begin[0] = false;
- }
- public boolean no_parentheses () {
- return parentheses[current] == 0;
- }
- public void new_context () {
- parentheses[++current] = 0;
- match_begin[current] = false;
- }
- public void close_context ( GenLex lex ) {
- if (parentheses[current--] > 0)
- lex.error("Unbalanced Parentheses in Gen construction/pattern");
- if (current < 0)
- lex.error("Wrong match statement");
- }
- public void open_parenthesis () {
- parentheses[current]++;
- }
- public boolean close_parenthesis () {
- return (--parentheses[current] == 0) && current > 0 && !match_begin[current];
- }
- public void in_match_body () {
- match_begin[current] = true;
- }
- public boolean match_body () { return match_begin[current]; }
-}
-%%
-%state gen
-%class GenLex
-%public
-%line
-%char
-%cup
-
-%eofval{
- return new java_cup.runtime.Symbol(GenSym.EOF);
-%eofval}
-
-%{
- public void error ( String msg ) {
- throw new Error("*** Lexical Error: " + msg + " (line: " + line_pos() + ", position: " + char_pos() + ")");
- }
-
- public static Context context = new Context();
-
- static int prev_char_pos = -1;
-
- public int line_pos () { return yyline+1; }
-
- public int char_pos () { return yychar-prev_char_pos; }
-
- public Symbol symbol ( int s ) {
- Tree.line_number = line_pos();
- Tree.position_number = char_pos();
- //System.out.println(context.parentheses[context.current]+" "+context.match_begin[context.current]+" "+GenParser.print(new Symbol(s)));
- return new Symbol(s);
- }
-
- public Symbol symbol ( int s, Object o ) {
- Tree.line_number = line_pos();
- Tree.position_number = char_pos();
- //System.out.println(context.parentheses[context.current]+" "+context.match_begin[context.current]+" "+GenParser.print(new Symbol(s,o)));
- return new Symbol(s,o);
- }
-%}
-
-DIGIT = [0-9]
-ID = [a-zA-Z_][a-zA-Z0-9_]*
-OPER = [!@#$%\^\&*-+=|\\~]+
-NEWLINE = [\n\r]
-DIGITS = {DIGIT}+
-INT = ({DIGIT}|[1-9]{DIGITS}|-{DIGIT}|-[1-9]{DIGITS})
-FRAC = [.]{DIGITS}
-EXP = [eE][+-]?{DIGITS}
-DOUBLE = ({INT}{FRAC}|{INT}{EXP}|{INT}{FRAC}{EXP})
-
-%%
-<gen> {INT} { return symbol(GenSym.LONG,new Long(yytext())); }
-<gen> {DOUBLE} { return symbol(GenSym.DOUBLE,new Double(yytext())); }
-<gen> ":" { yybegin(YYINITIAL);
- context.close_context(this);
- return symbol(GenSym.COLON); }
-<gen> "_" { return symbol(GenSym.ANY); }
-<gen> "," { return symbol(GenSym.COMMA); }
-<gen> "`(" { context.new_context();
- context.open_parenthesis();
- yybegin(YYINITIAL);
- return symbol(GenSym.BQP);
- }
-<gen> "...(" { context.new_context();
- context.open_parenthesis();
- yybegin(YYINITIAL);
- return symbol(GenSym.DOTSP);
- }
-<gen> "`" { return symbol(GenSym.BQ); }
-<gen> "..." { return symbol(GenSym.DOTS); }
-<gen> "(" { context.open_parenthesis();
- return symbol(GenSym.LP);
- }
-<gen> ")" { context.close_parenthesis();
- return symbol(GenSym.RP);
- }
-<gen> "[" { context.open_parenthesis();
- return symbol(GenSym.LB);
- }
-<gen> "]" { context.close_parenthesis();
- if (context.no_parentheses())
- { yybegin(YYINITIAL);
- context.close_context(this);
- };
- return symbol(GenSym.RB);
- }
-<gen> ">" { yybegin(YYINITIAL);
- context.close_parenthesis();
- context.close_context(this);
- return symbol(GenSym.GT);
- }
-<gen> "is" { return symbol(GenSym.IS); }
-<gen> {ID} { return symbol(GenSym.ID,yytext()); }
-<gen> {OPER} { return symbol(GenSym.OPER,yytext()); }
-<gen> \/\*[^*/]*\*\/ { for (char c: yytext().toCharArray())
- if (c=='\n' || c=='\r')
- GenParser.newlines++;
- prev_char_pos = yychar; }
-<gen> "//"[^\n\r]* { prev_char_pos = 0; }
-<gen> [ \t\f] {}
-<gen> {NEWLINE} { GenParser.newlines++; prev_char_pos = yychar; }
-<gen> . { error("Illegal character in Gen construction/pattern: "+yytext()); }
-<YYINITIAL> "match" { return symbol(GenSym.MATCH); }
-<YYINITIAL> "case" { if (!context.match_body())
- return symbol(GenSym.CHAR,yytext());
- context.new_context();
- yybegin(gen);
- return symbol(GenSym.CASE);
- }
-<YYINITIAL> "fail" { return symbol(GenSym.FAIL); }
-<YYINITIAL> "`" { error("Backquote outside a Gen construction/pattern"); }
-<YYINITIAL> "#<" { context.new_context();
- context.open_parenthesis();
- yybegin(gen);
- return symbol(GenSym.META);
- }
-<YYINITIAL> "#[" { context.new_context();
- context.open_parenthesis();
- yybegin(gen);
- return symbol(GenSym.METAL);
- }
-<YYINITIAL> "{" { context.open_parenthesis();
- if (context.match_body())
- return symbol(GenSym.LSB);
- else return symbol(GenSym.CHAR,yytext());
- }
-<YYINITIAL> "("|"[" { context.open_parenthesis();
- return symbol(GenSym.CHAR,yytext());
- }
-<YYINITIAL> "}" { context.close_parenthesis();
- if (context.match_body())
- return symbol(GenSym.RSB);
- else return symbol(GenSym.CHAR,yytext());
- }
-<YYINITIAL> ")" { if (context.close_parenthesis())
- { context.close_context(this);
- yybegin(gen);
- return symbol(GenSym.RP);
- } else return symbol(GenSym.CHAR,yytext());
- }
-<YYINITIAL> "]" { if (context.close_parenthesis())
- { context.close_context(this);
- yybegin(gen);
- return symbol(GenSym.RB);
- } else return symbol(GenSym.CHAR,yytext());
- }
-\"[^\"]*\" { return symbol(GenSym.CSTRING,yytext().substring(1,yytext().length()-1)); }
-<YYINITIAL> {ID} { return symbol(GenSym.CHAR,yytext()); }
-<YYINITIAL> {OPER} { return symbol(GenSym.CHAR,yytext()); }
-<YYINITIAL> \/\*[^*/]*\*\/ { prev_char_pos = yychar; return symbol(GenSym.CHAR,yytext()); }
-<YYINITIAL> "//"[^\n\r]* { prev_char_pos = 0; return symbol(GenSym.CHAR,yytext()); }
-<YYINITIAL> [ \t\f] { return symbol(GenSym.CHAR,yytext()); }
-<YYINITIAL> {NEWLINE} { prev_char_pos = yychar; return symbol(GenSym.CHAR,yytext()); }
-<YYINITIAL> . { return symbol(GenSym.CHAR,yytext()); }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/multipleinputs/DelegatingInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/multipleinputs/DelegatingInputFormat.java b/src/main/java/multipleinputs/DelegatingInputFormat.java
new file mode 100644
index 0000000..0d9b8f9
--- /dev/null
+++ b/src/main/java/multipleinputs/DelegatingInputFormat.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.lib.input;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * An {@link InputFormat} that delegates behavior of paths to multiple other
+ * InputFormats.
+ *
+ * @see MultipleInputs#addInputPath(Job, Path, Class, Class)
+ */
+public class DelegatingInputFormat<K, V> extends InputFormat<K, V> {
+
+ @SuppressWarnings("unchecked")
+ public List<InputSplit> getSplits(JobContext job)
+ throws IOException, InterruptedException {
+ Configuration conf = job.getConfiguration();
+ Job jobCopy =new Job(conf);
+ List<InputSplit> splits = new ArrayList<InputSplit>();
+ Map<Path, InputFormat> formatMap =
+ MultipleInputs.getInputFormatMap(job);
+ Map<Path, Class<? extends Mapper>> mapperMap = MultipleInputs
+ .getMapperTypeMap(job);
+ Map<Class<? extends InputFormat>, List<Path>> formatPaths
+ = new HashMap<Class<? extends InputFormat>, List<Path>>();
+
+ // First, build a map of InputFormats to Paths
+ for (Entry<Path, InputFormat> entry : formatMap.entrySet()) {
+ if (!formatPaths.containsKey(entry.getValue().getClass())) {
+ formatPaths.put(entry.getValue().getClass(), new LinkedList<Path>());
+ }
+
+ formatPaths.get(entry.getValue().getClass()).add(entry.getKey());
+ }
+
+ for (Entry<Class<? extends InputFormat>, List<Path>> formatEntry :
+ formatPaths.entrySet()) {
+ Class<? extends InputFormat> formatClass = formatEntry.getKey();
+ InputFormat format = (InputFormat) ReflectionUtils.newInstance(
+ formatClass, conf);
+ List<Path> paths = formatEntry.getValue();
+
+ Map<Class<? extends Mapper>, List<Path>> mapperPaths
+ = new HashMap<Class<? extends Mapper>, List<Path>>();
+
+ // Now, for each set of paths that have a common InputFormat, build
+ // a map of Mappers to the paths they're used for
+ for (Path path : paths) {
+ Class<? extends Mapper> mapperClass = mapperMap.get(path);
+ if (!mapperPaths.containsKey(mapperClass)) {
+ mapperPaths.put(mapperClass, new LinkedList<Path>());
+ }
+
+ mapperPaths.get(mapperClass).add(path);
+ }
+
+ // Now each set of paths that has a common InputFormat and Mapper can
+ // be added to the same job, and split together.
+ for (Entry<Class<? extends Mapper>, List<Path>> mapEntry :
+ mapperPaths.entrySet()) {
+ paths = mapEntry.getValue();
+ Class<? extends Mapper> mapperClass = mapEntry.getKey();
+
+ if (mapperClass == null) {
+ try {
+ mapperClass = job.getMapperClass();
+ } catch (ClassNotFoundException e) {
+ throw new IOException("Mapper class is not found", e);
+ }
+ }
+
+ FileInputFormat.setInputPaths(jobCopy, paths.toArray(new Path[paths
+ .size()]));
+
+ // Get splits for each input path and tag with InputFormat
+ // and Mapper types by wrapping in a TaggedInputSplit.
+ List<InputSplit> pathSplits = format.getSplits(jobCopy);
+ for (InputSplit pathSplit : pathSplits) {
+ splits.add(new TaggedInputSplit(pathSplit, conf, format.getClass(),
+ mapperClass));
+ }
+ }
+ }
+
+ return splits;
+ }
+
+ @Override
+ public RecordReader<K, V> createRecordReader(InputSplit split,
+ TaskAttemptContext context) throws IOException, InterruptedException {
+ return new DelegatingRecordReader<K, V>(split, context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/multipleinputs/DelegatingMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/multipleinputs/DelegatingMapper.java b/src/main/java/multipleinputs/DelegatingMapper.java
new file mode 100644
index 0000000..d72d2b5
--- /dev/null
+++ b/src/main/java/multipleinputs/DelegatingMapper.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.lib.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * An {@link Mapper} that delegates behavior of paths to multiple other
+ * mappers.
+ *
+ * @see MultipleInputs#addInputPath(Job, Path, Class, Class)
+ */
+public class DelegatingMapper<K1, V1, K2, V2> extends Mapper<K1, V1, K2, V2> {
+
+ private Mapper<K1, V1, K2, V2> mapper;
+
+ @SuppressWarnings("unchecked")
+ protected void setup(Context context)
+ throws IOException, InterruptedException {
+ // Find the Mapper from the TaggedInputSplit.
+ TaggedInputSplit inputSplit = (TaggedInputSplit) context.getInputSplit();
+ mapper = (Mapper<K1, V1, K2, V2>) ReflectionUtils.newInstance(inputSplit
+ .getMapperClass(), context.getConfiguration());
+
+ }
+
+ @SuppressWarnings("unchecked")
+ public void run(Context context)
+ throws IOException, InterruptedException {
+ setup(context);
+ mapper.run(context);
+ cleanup(context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/multipleinputs/DelegatingRecordReader.java
----------------------------------------------------------------------
diff --git a/src/main/java/multipleinputs/DelegatingRecordReader.java b/src/main/java/multipleinputs/DelegatingRecordReader.java
new file mode 100644
index 0000000..f0d060e
--- /dev/null
+++ b/src/main/java/multipleinputs/DelegatingRecordReader.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.lib.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This is a delegating RecordReader, which delegates the functionality to the
+ * underlying record reader in {@link TaggedInputSplit}
+ */
+public class DelegatingRecordReader<K, V> extends RecordReader<K, V> {
+ RecordReader<K, V> originalRR;
+
+ /**
+ * Constructs the DelegatingRecordReader.
+ *
+ * @param split TaggegInputSplit object
+ * @param context TaskAttemptContext object
+ *
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ @SuppressWarnings("unchecked")
+ public DelegatingRecordReader(InputSplit split, TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ // Find the InputFormat and then the RecordReader from the
+ // TaggedInputSplit.
+ TaggedInputSplit taggedInputSplit = (TaggedInputSplit) split;
+ InputFormat<K, V> inputFormat = (InputFormat<K, V>) ReflectionUtils
+ .newInstance(taggedInputSplit.getInputFormatClass(), context
+ .getConfiguration());
+ originalRR = inputFormat.createRecordReader(taggedInputSplit
+ .getInputSplit(), context);
+ }
+
+ @Override
+ public void close() throws IOException {
+ originalRR.close();
+ }
+
+ @Override
+ public K getCurrentKey() throws IOException, InterruptedException {
+ return originalRR.getCurrentKey();
+ }
+
+ @Override
+ public V getCurrentValue() throws IOException, InterruptedException {
+ return originalRR.getCurrentValue();
+ }
+
+ @Override
+ public float getProgress() throws IOException, InterruptedException {
+ return originalRR.getProgress();
+ }
+
+ @Override
+ public void initialize(InputSplit split, TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ originalRR.initialize(((TaggedInputSplit) split).getInputSplit(), context);
+ }
+
+ @Override
+ public boolean nextKeyValue() throws IOException, InterruptedException {
+ return originalRR.nextKeyValue();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/multipleinputs/MultipleInputs.java
----------------------------------------------------------------------
diff --git a/src/main/java/multipleinputs/MultipleInputs.java b/src/main/java/multipleinputs/MultipleInputs.java
new file mode 100644
index 0000000..9140512
--- /dev/null
+++ b/src/main/java/multipleinputs/MultipleInputs.java
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.lib.input;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This class supports MapReduce jobs that have multiple input paths with
+ * a different {@link InputFormat} and {@link Mapper} for each path
+ */
+public class MultipleInputs {
+ /**
+ * Add a {@link Path} with a custom {@link InputFormat} to the list of
+ * inputs for the map-reduce job.
+ *
+ * @param job The {@link Job}
+ * @param path {@link Path} to be added to the list of inputs for the job
+ * @param inputFormatClass {@link InputFormat} class to use for this path
+ */
+ @SuppressWarnings("unchecked")
+ public static void addInputPath(Job job, Path path,
+ Class<? extends InputFormat> inputFormatClass) {
+ String inputFormatMapping = path.toString() + ";"
+ + inputFormatClass.getName();
+ Configuration conf = job.getConfiguration();
+ String inputFormats = conf.get("mapred.input.dir.formats");
+ conf.set("mapred.input.dir.formats",
+ inputFormats == null ? inputFormatMapping : inputFormats + ","
+ + inputFormatMapping);
+
+ job.setInputFormatClass(DelegatingInputFormat.class);
+ }
+
+ /**
+ * Add a {@link Path} with a custom {@link InputFormat} and
+ * {@link Mapper} to the list of inputs for the map-reduce job.
+ *
+ * @param job The {@link Job}
+ * @param path {@link Path} to be added to the list of inputs for the job
+ * @param inputFormatClass {@link InputFormat} class to use for this path
+ * @param mapperClass {@link Mapper} class to use for this path
+ */
+ @SuppressWarnings("unchecked")
+ public static void addInputPath(Job job, Path path,
+ Class<? extends InputFormat> inputFormatClass,
+ Class<? extends Mapper> mapperClass) {
+
+ addInputPath(job, path, inputFormatClass);
+ Configuration conf = job.getConfiguration();
+ String mapperMapping = path.toString() + ";" + mapperClass.getName();
+ String mappers = conf.get("mapred.input.dir.mappers");
+ conf.set("mapred.input.dir.mappers", mappers == null ? mapperMapping
+ : mappers + "," + mapperMapping);
+
+ job.setMapperClass(DelegatingMapper.class);
+ }
+
+ /**
+ * Retrieves a map of {@link Path}s to the {@link InputFormat} class
+ * that should be used for them.
+ *
+ * @param job The {@link JobContext}
+ * @see #addInputPath(JobConf, Path, Class)
+ * @return A map of paths to inputformats for the job
+ */
+ @SuppressWarnings("unchecked")
+ static Map<Path, InputFormat> getInputFormatMap(JobContext job) {
+ Map<Path, InputFormat> m = new HashMap<Path, InputFormat>();
+ Configuration conf = job.getConfiguration();
+ String[] pathMappings = conf.get("mapred.input.dir.formats").split(",");
+ for (String pathMapping : pathMappings) {
+ String[] split = pathMapping.split(";");
+ InputFormat inputFormat;
+ try {
+ inputFormat = (InputFormat) ReflectionUtils.newInstance(conf
+ .getClassByName(split[1]), conf);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException(e);
+ }
+ m.put(new Path(split[0]), inputFormat);
+ }
+ return m;
+ }
+
+ /**
+ * Retrieves a map of {@link Path}s to the {@link Mapper} class that
+ * should be used for them.
+ *
+ * @param job The {@link JobContext}
+ * @see #addInputPath(JobConf, Path, Class, Class)
+ * @return A map of paths to mappers for the job
+ */
+ @SuppressWarnings("unchecked")
+ static Map<Path, Class<? extends Mapper>>
+ getMapperTypeMap(JobContext job) {
+ Configuration conf = job.getConfiguration();
+ if (conf.get("mapred.input.dir.mappers") == null) {
+ return Collections.emptyMap();
+ }
+ Map<Path, Class<? extends Mapper>> m =
+ new HashMap<Path, Class<? extends Mapper>>();
+ String[] pathMappings = conf.get("mapred.input.dir.mappers").split(",");
+ for (String pathMapping : pathMappings) {
+ String[] split = pathMapping.split(";");
+ Class<? extends Mapper> mapClass;
+ try {
+ mapClass =
+ (Class<? extends Mapper>) conf.getClassByName(split[1]);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException(e);
+ }
+ m.put(new Path(split[0]), mapClass);
+ }
+ return m;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/multipleinputs/TaggedInputSplit.java
----------------------------------------------------------------------
diff --git a/src/main/java/multipleinputs/TaggedInputSplit.java b/src/main/java/multipleinputs/TaggedInputSplit.java
new file mode 100644
index 0000000..68bb789
--- /dev/null
+++ b/src/main/java/multipleinputs/TaggedInputSplit.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.lib.input;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * An {@link InputSplit} that tags another InputSplit with extra data for use
+ * by {@link DelegatingInputFormat}s and {@link DelegatingMapper}s.
+ */
+class TaggedInputSplit extends InputSplit implements Configurable, Writable {
+
+ private Class<? extends InputSplit> inputSplitClass;
+
+ private InputSplit inputSplit;
+
+ @SuppressWarnings("unchecked")
+ private Class<? extends InputFormat> inputFormatClass;
+
+ @SuppressWarnings("unchecked")
+ private Class<? extends Mapper> mapperClass;
+
+ private Configuration conf;
+
+ public TaggedInputSplit() {
+ // Default constructor.
+ }
+
+ /**
+ * Creates a new TaggedInputSplit.
+ *
+ * @param inputSplit The InputSplit to be tagged
+ * @param conf The configuration to use
+ * @param inputFormatClass The InputFormat class to use for this job
+ * @param mapperClass The Mapper class to use for this job
+ */
+ @SuppressWarnings("unchecked")
+ public TaggedInputSplit(InputSplit inputSplit, Configuration conf,
+ Class<? extends InputFormat> inputFormatClass,
+ Class<? extends Mapper> mapperClass) {
+ this.inputSplitClass = inputSplit.getClass();
+ this.inputSplit = inputSplit;
+ this.conf = conf;
+ this.inputFormatClass = inputFormatClass;
+ this.mapperClass = mapperClass;
+ }
+
+ /**
+ * Retrieves the original InputSplit.
+ *
+ * @return The InputSplit that was tagged
+ */
+ public InputSplit getInputSplit() {
+ return inputSplit;
+ }
+
+ /**
+ * Retrieves the InputFormat class to use for this split.
+ *
+ * @return The InputFormat class to use
+ */
+ @SuppressWarnings("unchecked")
+ public Class<? extends InputFormat> getInputFormatClass() {
+ return inputFormatClass;
+ }
+
+ /**
+ * Retrieves the Mapper class to use for this split.
+ *
+ * @return The Mapper class to use
+ */
+ @SuppressWarnings("unchecked")
+ public Class<? extends Mapper> getMapperClass() {
+ return mapperClass;
+ }
+
+ public long getLength() throws IOException, InterruptedException {
+ return inputSplit.getLength();
+ }
+
+ public String[] getLocations() throws IOException, InterruptedException {
+ return inputSplit.getLocations();
+ }
+
+ @SuppressWarnings("unchecked")
+ public void readFields(DataInput in) throws IOException {
+ inputSplitClass = (Class<? extends InputSplit>) readClass(in);
+ inputFormatClass = (Class<? extends InputFormat<?, ?>>) readClass(in);
+ mapperClass = (Class<? extends Mapper<?, ?, ?, ?>>) readClass(in);
+ inputSplit = (InputSplit) ReflectionUtils
+ .newInstance(inputSplitClass, conf);
+ SerializationFactory factory = new SerializationFactory(conf);
+ Deserializer deserializer = factory.getDeserializer(inputSplitClass);
+ deserializer.open((DataInputStream)in);
+ inputSplit = (InputSplit)deserializer.deserialize(inputSplit);
+ }
+
+ private Class<?> readClass(DataInput in) throws IOException {
+ String className = Text.readString(in);
+ try {
+ return conf.getClassByName(className);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("readObject can't find class", e);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void write(DataOutput out) throws IOException {
+ Text.writeString(out, inputSplitClass.getName());
+ Text.writeString(out, inputFormatClass.getName());
+ Text.writeString(out, mapperClass.getName());
+ SerializationFactory factory = new SerializationFactory(conf);
+ Serializer serializer =
+ factory.getSerializer(inputSplitClass);
+ serializer.open((DataOutputStream)out);
+ serializer.serialize(inputSplit);
+ }
+
+ public Configuration getConf() {
+ return conf;
+ }
+
+ public void setConf(Configuration conf) {
+ this.conf = conf;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/MR_rdd.java
----------------------------------------------------------------------
diff --git a/src/main/java/spark/MR_rdd.java b/src/main/java/spark/MR_rdd.java
deleted file mode 100644
index 48f4f92..0000000
--- a/src/main/java/spark/MR_rdd.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.spark.api.java.JavaRDD;
-
-
-/** a wrapper of a JavaRDD<MRData> (stored in HDFS) as an MRData */
-final public class MR_rdd extends MRData {
- public JavaRDD<MRData> rdd;
-
- public MR_rdd ( JavaRDD<MRData> d ) { rdd = d; }
-
- public void materializeAll () {};
-
- public JavaRDD<MRData> rdd () { return rdd; }
-
- final public void write ( DataOutput out ) throws IOException {
- throw new Error("RDDs are not serializable");
- }
-
- public void readFields ( DataInput in ) throws IOException {
- throw new Error("RDDs are not serializable");
- }
-
- public int compareTo ( MRData x ) {
- throw new Error("RDDs cannot be compared");
- }
-
- public boolean equals ( Object x ) {
- throw new Error("RDDs cannot be compared");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/RDDDataSource.java
----------------------------------------------------------------------
diff --git a/src/main/java/spark/RDDDataSource.java b/src/main/java/spark/RDDDataSource.java
deleted file mode 100644
index b0345da..0000000
--- a/src/main/java/spark/RDDDataSource.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.util.List;
-import java.io.Serializable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.Function2;
-
-
-final public class RDDDataSource extends DataSource implements Serializable {
- public JavaRDD<MRData> rdd;
-
- RDDDataSource ( JavaRDD<MRData> rdd ) {
- super();
- this.rdd = rdd;
- }
-
- @Override
- public long size ( Configuration conf ) {
- return rdd.count();
- }
-
- /** return the first num values */
- @Override
- public List<MRData> take ( int num ) {
- return (num < 0) ? rdd.collect() : rdd.take(num);
- }
-
- /** accumulate all dataset values */
- @Override
- public MRData reduce ( final MRData zero, final Function acc ) {
- return rdd.aggregate(zero,new Function2<MRData,MRData,MRData>() {
- Tuple t = new Tuple(new Tuple(),new Tuple());
- public MRData call ( MRData x, MRData y ) {
- t.set(0,x).set(1,y);
- return acc.eval(t);
- }
- },new Function2<MRData,MRData,MRData>() {
- Tuple t = new Tuple(new Tuple(),new Tuple());
- public MRData call ( MRData x, MRData y ) {
- return (y.equals(zero)) ? x : y;
- }
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/spark/SparkBinaryInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/spark/SparkBinaryInputFormat.java b/src/main/java/spark/SparkBinaryInputFormat.java
deleted file mode 100644
index f11c796..0000000
--- a/src/main/java/spark/SparkBinaryInputFormat.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
-
-
-/** Input format for hadoop sequence files */
-final public class SparkBinaryInputFormat extends SparkMRQLFileInputFormat {
- public static class BinaryInputRecordReader extends SequenceFileRecordReader<MRContainer,MRContainer> {
- final MRContainer result = new MRContainer();
-
- public BinaryInputRecordReader ( FileSplit split,
- JobConf job ) throws IOException {
- super(job,split);
- }
-
- @Override
- public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
- boolean b = super.next(key,result);
- value.set(result.data());
- return b;
- }
- }
-
- @Override
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split,
- JobConf job,
- Reporter reporter ) throws IOException {
- String path = ((FileSplit)split).getPath().toString();
- BinaryDataSource ds = (BinaryDataSource)DataSource.get(path,job);
- return new BinaryInputRecordReader((FileSplit)split,job);
- }
-}
[14/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReducePlan.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReducePlan.java b/mapreduce/src/main/java/org/apache/mrql/MapReducePlan.java
new file mode 100644
index 0000000..ae243cd
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReducePlan.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+
+
+/** A superclass for Hadoop MapReduce physical operators (files *Operator.java) */
+public class MapReducePlan extends Plan {
+
+ MapReducePlan () {}
+
+ /** find the number of records in the hadoop MapReduce job output */
+ public final static long outputRecords ( Job job ) throws Exception {
+ CounterGroup cg = job.getCounters().getGroup("org.apache.hadoop.mapred.Task$Counter");
+ long rc = cg.findCounter("REDUCE_OUTPUT_RECORDS").getValue();
+ if (rc == 0)
+ return cg.findCounter("MAP_OUTPUT_RECORDS").getValue();
+ return rc;
+ }
+
+ /** The Aggregate physical operator
+ * @param acc_fnc the accumulator function from (T,T) to T
+ * @param zero the zero element of type T
+ * @param S the dataset that contains the bag of values {T}
+ * @return the aggregation result of type T
+ */
+ public final static MRData aggregate ( final Tree acc_fnc,
+ final Tree zero,
+ final DataSet S ) throws Exception {
+ MRData res = Interpreter.evalE(zero);
+ Function accumulator = functional_argument(Plan.conf,acc_fnc);
+ Tuple pair = new Tuple(2);
+ for ( DataSource s: S.source )
+ if (s.inputFormat != MapReduceBinaryInputFormat.class) {
+ pair.set(0,res);
+ pair.set(1,aggregate(acc_fnc,zero,
+ MapOperation.cMap(Interpreter.identity_mapper,acc_fnc,zero,
+ new DataSet(s,0,0),"-")));
+ res = accumulator.eval(pair);
+ } else {
+ Path path = new Path(s.path);
+ final FileSystem fs = path.getFileSystem(conf);
+ final FileStatus[] ds
+ = fs.listStatus(path,
+ new PathFilter () {
+ public boolean accept ( Path path ) {
+ return !path.getName().startsWith("_");
+ }
+ });
+ MRContainer key = new MRContainer(new MR_int(0));
+ MRContainer value = new MRContainer(new MR_int(0));
+ for ( int i = 0; i < ds.length; i++ ) {
+ SequenceFile.Reader reader = new SequenceFile.Reader(fs,ds[i].getPath(),conf);
+ while (reader.next(key,value)) {
+ pair.set(0,res);
+ pair.set(1,value.data());
+ res = accumulator.eval(pair);
+ };
+ reader.close();
+ }
+ };
+ return res;
+ }
+
+ /** The repeat physical operator. It repeats the loop until all values satisfy
+ * the termination condition (when dataset.counter == 0)
+ * @param loop the function from DataSet to DataSet to be repeated
+ * @param init the initial input DataSet for the loop
+ * @param max_num max number of repetitions
+ * @return the resulting DataSet from the loop
+ */
+ public final static DataSet repeat ( Function loop, DataSet init, int max_num ) {
+ MR_dataset s = new MR_dataset(init);
+ int i = 0;
+ do {
+ s.dataset = ((MR_dataset)loop.eval(s)).dataset;
+ i++;
+ System.err.println("*** Repeat #"+i+": "+s.dataset.counter+" true results");
+ } while (s.dataset.counter != 0 && i < max_num);
+ return s.dataset;
+ }
+
+ /** The closure physical operator. It repeats the loop until the size
+ * of the new DataSet is equal to that of the previous DataSet
+ * @param loop the function from DataSet to DataSet to be repeated
+ * @param init the initial input DataSet for the loop
+ * @param max_num max number of repetitions
+ * @return the resulting DataSet from the loop
+ */
+ public final static DataSet closure ( Function loop, DataSet init, int max_num ) {
+ MR_dataset s = new MR_dataset(init);
+ int i = 0;
+ long n = 0;
+ long old = 0;
+ do {
+ s.dataset = ((MR_dataset)loop.eval(s)).dataset;
+ i++;
+ System.err.println("*** Repeat #"+i+": "+(s.dataset.records-n)+" new records");
+ old = n;
+ n = s.dataset.records;
+ } while (old < n && i < max_num);
+ return s.dataset;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dd680aa..5761c54 100644
--- a/pom.xml
+++ b/pom.xml
@@ -52,11 +52,11 @@
</properties>
<modules>
- <module>Gen</module>
+ <module>gen</module>
<module>core</module>
- <module>MapReduce</module>
- <module>BSP</module>
- <module>Spark</module>
+ <module>mapreduce</module>
+ <module>bsp</module>
+ <module>spark</module>
<module>dist</module>
</modules>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/pom.xml
----------------------------------------------------------------------
diff --git a/spark/pom.xml b/spark/pom.xml
new file mode 100644
index 0000000..c0096e4
--- /dev/null
+++ b/spark/pom.xml
@@ -0,0 +1,151 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-spark</artifactId>
+ <packaging>jar</packaging>
+ <name>Apache MRQL Spark mode</name>
+ <description>Apache MRQL evaluation in Spark mode on Apache Spark</description>
+ <url>http://mrql.incubator.apache.org/</url>
+ <inceptionYear>2013</inceptionYear>
+
+ <parent>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-parent</artifactId>
+ <version>0.9.1-incubating-SNAPSHOT</version>
+ </parent>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-gen</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-core_${scala.version}</artifactId>
+ <version>${spark.version}</version>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.8</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals><goal>add-source</goal></goals>
+ <configuration>
+ <sources>
+ <source>${project.build.directory}/generated-sources</source>
+ </sources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>2.1</version>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <artifactSet>
+ <includes>
+ <include>org.apache.mrql:*</include>
+ </includes>
+ <excludes>
+ <exclude>org.apache.mrql:mrql-gen</exclude>
+ <exclude>org.apache.mrql:mrql-core</exclude>
+ </excludes>
+ </artifactSet>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.4</version>
+ <configuration>
+ <finalName>mrql-spark-${project.version}</finalName>
+ <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <version>1.7</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target>
+ <mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
+ <property name="compile_classpath" refid="maven.compile.classpath" />
+ <fileset id="mr.gen.path" dir="src/main/java/org/apache/mrql" includes="*.gen" />
+ <pathconvert pathsep=" " property="mr.gen.files" refid="mr.gen.path" />
+ <java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
+ <arg line="${mr.gen.files}" />
+ <arg line="-o" />
+ <arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
+ </java>
+ </target>
+ </configuration>
+ </execution>
+ <execution>
+ <id>validate</id>
+ <phase>test</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target name="validate_spark" if="tests" description="Validate all test queries on Apache Spark">
+ <property name="runtime_classpath" refid="maven.runtime.classpath" />
+ <echo message="Evaluating test queries in Apache Spark local mode:" />
+ <java classname="org.apache.mrql.Test" classpath="../lib/mrql-spark-${project.version}.jar:${runtime_classpath}" dir=".." fork="yes" error="/dev/null">
+ <arg line="-local" />
+ <arg line="-spark" />
+ <arg file="../tests/queries" />
+ <arg file="../tests/results/spark" />
+ <arg file="../tests/error_log.txt" />
+ </java>
+ </target>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/MR_rdd.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/MR_rdd.java b/spark/src/main/java/org/apache/mrql/MR_rdd.java
new file mode 100644
index 0000000..48f4f92
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/MR_rdd.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.spark.api.java.JavaRDD;
+
+
+/** a wrapper of a JavaRDD<MRData> (stored in HDFS) as an MRData */
+final public class MR_rdd extends MRData {
+ public JavaRDD<MRData> rdd;
+
+ public MR_rdd ( JavaRDD<MRData> d ) { rdd = d; }
+
+ public void materializeAll () {};
+
+ public JavaRDD<MRData> rdd () { return rdd; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ throw new Error("RDDs are not serializable");
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ throw new Error("RDDs are not serializable");
+ }
+
+ public int compareTo ( MRData x ) {
+ throw new Error("RDDs cannot be compared");
+ }
+
+ public boolean equals ( Object x ) {
+ throw new Error("RDDs cannot be compared");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/RDDDataSource.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/RDDDataSource.java b/spark/src/main/java/org/apache/mrql/RDDDataSource.java
new file mode 100644
index 0000000..b0345da
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/RDDDataSource.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.util.List;
+import java.io.Serializable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function2;
+
+
+final public class RDDDataSource extends DataSource implements Serializable {
+ public JavaRDD<MRData> rdd;
+
+ RDDDataSource ( JavaRDD<MRData> rdd ) {
+ super();
+ this.rdd = rdd;
+ }
+
+ @Override
+ public long size ( Configuration conf ) {
+ return rdd.count();
+ }
+
+ /** return the first num values */
+ @Override
+ public List<MRData> take ( int num ) {
+ return (num < 0) ? rdd.collect() : rdd.take(num);
+ }
+
+ /** accumulate all dataset values */
+ @Override
+ public MRData reduce ( final MRData zero, final Function acc ) {
+ return rdd.aggregate(zero,new Function2<MRData,MRData,MRData>() {
+ Tuple t = new Tuple(new Tuple(),new Tuple());
+ public MRData call ( MRData x, MRData y ) {
+ t.set(0,x).set(1,y);
+ return acc.eval(t);
+ }
+ },new Function2<MRData,MRData,MRData>() {
+ Tuple t = new Tuple(new Tuple(),new Tuple());
+ public MRData call ( MRData x, MRData y ) {
+ return (y.equals(zero)) ? x : y;
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/SparkBinaryInputFormat.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/SparkBinaryInputFormat.java b/spark/src/main/java/org/apache/mrql/SparkBinaryInputFormat.java
new file mode 100644
index 0000000..f11c796
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/SparkBinaryInputFormat.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+
+
+/** Input format for hadoop sequence files */
+final public class SparkBinaryInputFormat extends SparkMRQLFileInputFormat {
+ public static class BinaryInputRecordReader extends SequenceFileRecordReader<MRContainer,MRContainer> {
+ final MRContainer result = new MRContainer();
+
+ public BinaryInputRecordReader ( FileSplit split,
+ JobConf job ) throws IOException {
+ super(job,split);
+ }
+
+ @Override
+ public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
+ boolean b = super.next(key,result);
+ value.set(result.data());
+ return b;
+ }
+ }
+
+ @Override
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split,
+ JobConf job,
+ Reporter reporter ) throws IOException {
+ String path = ((FileSplit)split).getPath().toString();
+ BinaryDataSource ds = (BinaryDataSource)DataSource.get(path,job);
+ return new BinaryInputRecordReader((FileSplit)split,job);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/SparkEvaluator.gen
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/SparkEvaluator.gen b/spark/src/main/java/org/apache/mrql/SparkEvaluator.gen
new file mode 100644
index 0000000..611eca2
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/SparkEvaluator.gen
@@ -0,0 +1,762 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java_cup.runtime.Scanner;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.io.*;
+import java.util.Enumeration;
+import org.apache.log4j.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import scala.Tuple2;
+import org.apache.spark.TaskContext;
+import org.apache.spark.Partition;
+import org.apache.spark.Accumulator;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.VoidFunction;
+
+
+/** Evaluates physical plans in Apache Spark mode */
+final public class SparkEvaluator extends Evaluator implements Serializable {
+ public static JavaSparkContext spark_context;
+ final static String spark_jar = SparkEvaluator.class.getProtectionDomain().getCodeSource().getLocation().toString();
+ final static String core_jar = Interpreter.class.getProtectionDomain().getCodeSource().getLocation().toString();
+ final static String gen_jar = Tree.class.getProtectionDomain().getCodeSource().getLocation().toString();
+ final static String scanner_jar = Scanner.class.getProtectionDomain().getCodeSource().getLocation().toString();
+ static String data_source_directory;
+ final static Bag empty_bag = new Bag();
+
+ /** initialize the Spark evaluator */
+ final public void init ( Configuration conf ) {
+ Config.spark_mode = true;
+ try {
+ if (Config.hadoop_mode && Config.local_mode)
+ spark_context = new JavaSparkContext("local["+Config.nodes+"]",
+ "Apache MRQL Local Spark Evaluator");
+ else if (Config.hadoop_mode) {
+ HashMap<String,String> env = new HashMap<String,String>();
+ data_source_directory = System.getenv("FS_DEFAULT_NAME")+"/"+Plan.new_path(conf);
+ env.put("mrql.data.source.directory",data_source_directory);
+ spark_context = new JavaSparkContext(System.getenv("SPARK_MASTER"),
+ "Apache MRQL Spark Evaluator",
+ System.getenv("SPARK_HOME"),
+ new String[]{spark_jar,core_jar,gen_jar,scanner_jar},
+ env);
+ Plan.conf = spark_context.hadoopConfiguration();
+ FileSystem.setDefaultUri(Plan.conf,System.getenv("FS_DEFAULT_NAME"));
+ };
+ if (!Config.info) {
+ for ( Enumeration en = LogManager.getCurrentLoggers(); en.hasMoreElements(); )
+ ((Logger)en.nextElement()).setLevel(Level.WARN);
+ LogManager.getRootLogger().setLevel(Level.WARN);
+ }
+ } catch (IOException ex) {
+ throw new Error(ex);
+ }
+ }
+
+ /** shutdown the Spark evaluator */
+ final public void shutdown ( Configuration conf ) {
+ spark_context.stop();
+ spark_context = null;
+ System.clearProperty("spark.driver.port");
+ }
+
+ final public void initialize_query () {
+ Plan.distribute_compiled_arguments(Plan.conf);
+ if (Config.compile_functional_arguments)
+ spark_context.addJar(Plan.conf.get("mrql.jar.path"));
+ }
+
+ final public Configuration new_configuration () {
+ return new Configuration();
+ }
+
+ /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
+ final public Class<? extends MRQLFileInputFormat> parsedInputFormat () {
+ return SparkParsedInputFormat.class;
+ }
+
+ /** return the FileInputFormat for binary files */
+ final public Class<? extends MRQLFileInputFormat> binaryInputFormat () {
+ return SparkBinaryInputFormat.class;
+ }
+
+ /** return the FileInputFormat for data generator files */
+ final public Class<? extends MRQLFileInputFormat> generatorInputFormat () {
+ return SparkGeneratorInputFormat.class;
+ }
+
+ /** used by the master to send parsing details (eg, record types) to workers */
+ public static void dump_source_dir () throws IOException {
+ if (Config.local_mode)
+ return;
+ DataSource.dataSourceDirectory.distribute(Plan.conf);
+ Path path = new Path(data_source_directory);
+ FileSystem fs = path.getFileSystem(Plan.conf);
+ PrintStream ps = new PrintStream(fs.create(path,true));
+ ps.println(Plan.conf.get("mrql.data.source.directory"));
+ ps.close();
+ }
+
+ /** executed by a worker when reading parsed input (see SparkParsedInputFormat) */
+ public static void load_source_dir () throws IOException {
+ if (Config.local_mode)
+ return;
+ if (Plan.conf == null) {
+ if (evaluator == null)
+ evaluator = new SparkEvaluator();
+ Plan.conf = evaluator.new_configuration();
+ };
+ // the name of the file that contains the source directory details is passed
+ // to workers through the HashMap environment in the JavaSparkContext
+ Path path = new Path(System.getenv("mrql.data.source.directory"));
+ FileSystem fs = path.getFileSystem(Plan.conf);
+ BufferedReader ftp = new BufferedReader(new InputStreamReader(fs.open(path)));
+ Plan.conf.set("mrql.data.source.directory",ftp.readLine());
+ DataSource.dataSourceDirectory.read(Plan.conf);
+ ftp.close();
+ }
+
+ /** dump MRQL data into a sequence file */
+ @Override
+ final public void dump ( String file, Tree type, MRData data ) throws Exception {
+ if (data instanceof MR_dataset && ((MR_dataset)data).dataset().source.get(0) instanceof RDDDataSource) {
+ Path path = new Path(file);
+ FileSystem fs = path.getFileSystem(Plan.conf);
+ PrintStream ftp = new PrintStream(fs.create(path.suffix(".type")));
+ fs.delete(path,true);
+ ftp.print("2@"+type.toString()+"\n");
+ ftp.close();
+ // in Spark mode, a dataset has always one RDD datasource
+ JavaRDD<MRData> rdd = ((RDDDataSource)((MR_dataset)data).dataset().source.get(0)).rdd.cache();
+ rdd.map(new PairFunction<MRData,MRContainer,MRContainer>() {
+ final MRContainer zero = new MRContainer(new MR_int(0));
+ public Tuple2<MRContainer,MRContainer> call ( MRData value ) {
+ return new Tuple2<MRContainer,MRContainer>(zero,new MRContainer(value));
+ }
+ })
+ .saveAsHadoopFile(file,MRContainer.class,MRContainer.class,SequenceFileOutputFormat.class);
+ } else super.dump(file,type,data);
+ }
+
+ /** dump MRQL data into text a CVS file */
+ @Override
+ final public void dump_text ( String file, Tree type, MRData data ) throws Exception {
+ if (data instanceof MR_dataset && ((MR_dataset)data).dataset().source.get(0) instanceof RDDDataSource) {
+ // in Spark mode, a dataset has always one RDD datasource
+ JavaRDD<MRData> rdd = ((RDDDataSource)((MR_dataset)data).dataset().source.get(0)).rdd.cache();
+ int ps = Config.max_bag_size_print;
+ Config.max_bag_size_print = -1;
+ match type {
+ case `T(tuple(...tps)):
+ final Trees ts = tps;
+ rdd.map(new Function<MRData,String>() {
+ public String call ( MRData value ) {
+ Tuple t = (Tuple)value;
+ String s = Printer.print(t.get((short)0),ts.nth(0));
+ for ( short i = 1; i < t.size(); i++ )
+ s += ","+Printer.print(t.get(i),ts.nth(i));
+ return s;
+ }
+ }).saveAsTextFile(file);
+ case _:
+ rdd.saveAsTextFile(file);
+ };
+ Config.max_bag_size_print = ps;
+ } else super.dump_text(file,type,data);
+ }
+
+ private static Function2<MRData,MRData,MRData> accumulator ( final Tree acc_fnc, final Environment env ) {
+ final org.apache.mrql.Function f = evalF(acc_fnc,env);
+ return new Function2<MRData,MRData,MRData>() {
+ public MRData call ( MRData x, MRData y ) {
+ return f.eval(new Tuple(x,y));
+ }
+ };
+ }
+
+ /** The Aggregate physical operator
+ * @param acc_fnc the accumulator function from (T,T) to T
+ * @param zero the zero element of type T
+ * @param plan the plan that constructs the dataset that contains the bag of values {T}
+ * @param env contains bindings fro variables to values (MRData)
+ * @return the aggregation result of type T
+ */
+ final public MRData aggregate ( Tree acc_fnc,
+ Tree zero,
+ Tree plan,
+ Environment env ) throws Exception {
+ Function2<MRData,MRData,MRData> f2 = accumulator(acc_fnc,env);
+ MRData z = evalE(zero,env);
+ match plan {
+ case AggregateMap(`m,`acc,_,`s):
+ return evalD(#<cMap(`m,`s)>,env)
+ .aggregate(z,accumulator(acc,env),f2);
+ case MapAggregateReduce(`m,`r,`acc,_,`s,`o):
+ if (acc.equals(#<null>))
+ fail;
+ return evalD(#<MapReduce(`m,`r,`s,`o)>,env)
+ .aggregate(z,accumulator(acc,env),f2);
+ case CrossAggregateProduct(`mx,`my,`r,`acc,_,`x,`y):
+ if (acc.equals(#<null>))
+ fail;
+ return evalD(#<CrossProduct(`mx,`my,`r,`x,`y)>,env)
+ .aggregate(z,accumulator(acc,env),f2);
+ case MapAggregateReduce2(`mx,`my,`r,`acc,_,`x,`y,`o):
+ if (acc.equals(#<null>))
+ fail;
+ return evalD(#<MapReduce2(`mx,`my,`r,`x,`y,`o)>,env)
+ .aggregate(z,accumulator(acc,env),f2);
+ case MapAggregateJoin(`mx,`my,`r,`acc,_,`x,`y):
+ if (acc.equals(#<null>))
+ fail;
+ return evalD(#<MapJoin(`mx,`my,`r,`x,`y)>,env)
+ .aggregate(z,accumulator(acc,env),f2);
+ };
+ throw new Error("Unrecognized aggregation: "+plan);
+ }
+
+ /** Evaluate a loop a fixed # of times */
+ final public Tuple loop ( Tree e, Environment env ) throws Exception {
+ match e {
+ case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`num):
+ int limit = ((MR_int)evalE(num,env)).get();
+ MR_rdd[] s = new MR_rdd[vs.length()];
+ for ( int i = 0; i < vs.length(); i++ )
+ s[i] = new MR_rdd(eval(ss.nth(i),env));
+ for ( int n = 0; n < limit; n++ ) {
+ Environment nenv = env;
+ for ( int i = 0; i < vs.length(); i ++ )
+ nenv = new Environment(vs.nth(i).toString(),s[i],nenv);
+ for ( int i = 0; i < vs.length(); i ++ )
+ s[i] = new MR_rdd(eval(bs.nth(i),nenv));
+ };
+ return new Tuple(s);
+ };
+ throw new Error("Wrong Loop format");
+ }
+
+ private static Bag bag ( final List<MRData> s ) {
+ final Iterator<MRData> i = s.iterator();
+ return new Bag(new BagIterator() {
+ public MRData next () {
+ return i.next();
+ }
+ public boolean hasNext () {
+ return i.hasNext();
+ }
+ });
+ }
+
+ // changed since Spark 0.8.1
+ final static TaskContext context = new TaskContext(0,0,(long)0,Config.local_mode,false,null);
+
+ /** Convert a Spark RDD into a lazy bag
+ * @param rdd the Spark RDD
+ * @return a lazy bag that contains all RDD elements
+ */
+ public static Bag bag ( final JavaRDD<MRData> rdd ) {
+ final JavaRDD<MRData> rd = rdd.cache();
+ rd.count(); // force the evaluation of all pending Spark operations
+ final List<Partition> ps = rd.splits();
+ return new Bag(new BagIterator() {
+ Iterator<MRData> i = null;
+ int c = 0;
+ public MRData next () {
+ return i.next();
+ }
+ public boolean hasNext () {
+ do {
+ if (i != null && i.hasNext())
+ return true;
+ if (c >= ps.size())
+ return false;
+ i = rdd.iterator(ps.get(c++),context);
+ } while (true);
+ }
+ });
+ }
+
+ private final static Function<MRData,MRData> get_first
+ = new Function<MRData,MRData>() {
+ public MRData call ( MRData value ) {
+ return ((Tuple)value).first();
+ };
+ };
+
+ /** Evaluate an MRQL physical plan using Spark
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @param counter the name of the counter used in loops
+ * @return a DataSet
+ */
+ final public DataSet eval ( final Tree e,
+ final Environment env,
+ final String counter ) {
+ JavaRDD<MRData> rd = eval(e,env);
+ int count = 0;
+ if (!counter.equals("-")) {
+ final Accumulator<Integer> c = spark_context.intAccumulator(0);
+ rd = rd.cache();
+ rd.foreach(new VoidFunction<MRData>() {
+ public void call ( MRData value ) {
+ if (((MR_bool)((Tuple)value).second()).get())
+ c.add(1); // count the true results (the results that need another loop step)
+ }
+ });
+ count = c.value();
+ rd = rd.map(get_first);
+ };
+ return new DataSet(new RDDDataSource(rd),count,0);
+ }
+
+ /** Evaluate an MRQL physical plan using Spark and print tracing info
+ * @param e the physical plan
+ * @param env contains bindings from variables to values (MRData)
+ * @return a Spark RDD
+ */
+ final public JavaRDD<MRData> eval ( final Tree e, final Environment env ) {
+ if (Config.trace_execution) {
+ tab_count += 3;
+ System.out.println(tabs(tab_count)+print_query(e));
+ };
+ final JavaRDD<MRData> res = evalD(e,env);
+ if (Config.trace_execution)
+ try {
+ System.out.println(tabs(tab_count)+"-> "+res.take(Config.max_bag_size_print));
+ tab_count -= 3;
+ } catch (Exception ex) {
+ throw new Error("Cannot collect the operator output: "+e);
+ };
+ return res;
+ }
+
+ /* convert an MRQL lambda to a Spark Function */
+ private static FlatMapFunction<MRData,MRData> cmap_fnc ( Tree fnc, Environment env ) {
+ final org.apache.mrql.Function f = evalF(fnc,env);
+ return new FlatMapFunction<MRData,MRData>() {
+ public Iterable<MRData> call ( MRData value ) {
+ return (Bag)f.eval(value);
+ }
+ };
+ }
+
+ /* group-by s and then reduce by fnc; if o is true, sort after group-by */
+ private static JavaRDD<MRData> groupBy ( JavaRDD<MRData> s, Tree fnc, Environment env, Tree o ) {
+ match o {
+ case true: // the result must be sorted
+ return s.groupBy(get_first)
+ .sortByKey()
+ .map(new Function<Tuple2<MRData,List<MRData>>,MRData>() {
+ public MRData call ( Tuple2<MRData,List<MRData>> value ) {
+ return new Tuple(value._1,bag(value._2));
+ }
+ })
+ .flatMap(cmap_fnc(fnc,env)).map(new Function<MRData,MRData>() {
+ public MRData call ( MRData value ) {
+ return ((Tuple)value).second();
+ }
+ });
+ };
+ return s.groupBy(get_first)
+ .map(new Function<Tuple2<MRData,List<MRData>>,MRData> () {
+ public MRData call ( Tuple2<MRData,List<MRData>> value ) {
+ final Iterator<MRData> i = value._2.iterator();
+ return new Tuple(value._1,new Bag(new BagIterator() {
+ public MRData next () {
+ return ((Tuple)i.next()).second();
+ }
+ public boolean hasNext () {
+ return i.hasNext();
+ }
+ }));
+ }
+ })
+ .flatMap(cmap_fnc(fnc,env));
+ }
+
+ private static JavaRDD<MRData> containerData ( JavaPairRDD<MRContainer,MRContainer> rd ) {
+ final Environment env = Interpreter.global_env;
+ return rd.map(new Function<Tuple2<MRContainer,MRContainer>,MRData>() {
+ boolean first = true;
+ public MRData call ( Tuple2<MRContainer,MRContainer> value ) {
+ if (first) {
+ // need to pass the global bindings (the in-memory repeat vars) to workers
+ Interpreter.set_global_bindings(env);
+ first = false;
+ };
+ return value._2.data();
+ }
+ });
+ }
+
+ private static Iterable<Tuple2<MRData,MRData>> joinIterator ( final Iterator<MRData> i ) {
+ return new Iterable<Tuple2<MRData,MRData>>() {
+ public Iterator<Tuple2<MRData,MRData>> iterator() {
+ return new Iterator<Tuple2<MRData,MRData>> () {
+ public Tuple2<MRData,MRData> next () {
+ Tuple data = (Tuple)i.next();
+ return new Tuple2<MRData,MRData>(data.first(),data.second());
+ }
+ public boolean hasNext () {
+ return i.hasNext();
+ }
+ public void remove () {}
+ };
+ }
+ };
+ }
+
+ private static FlatMapFunction<Iterator<MRData>,MRData> combiner_fnc ( final org.apache.mrql.Function f ) {
+ return new FlatMapFunction<Iterator<MRData>,MRData>() {
+ public Iterable<MRData> call ( final Iterator<MRData> i ) {
+ return MapReduceAlgebra.cmap(new org.apache.mrql.Function() {
+ public MRData eval ( MRData x ) {
+ final MRData key = ((Tuple)x).first();
+ final Iterator<MRData> it = ((Bag)f.eval(x)).iterator();
+ return new Bag(new BagIterator() {
+ public MRData next () {
+ return new Tuple(key,it.next());
+ }
+ public boolean hasNext () {
+ return it.hasNext();
+ }
+ });
+ }},
+ MapReduceAlgebra.groupBy(new Bag(new BagIterator() {
+ public MRData next () {
+ return i.next();
+ }
+ public boolean hasNext () {
+ return i.hasNext();
+ }
+ })));
+ }
+ };
+ }
+
+ private static Hashtable<MRData,Bag> make_built_table ( List<Tuple2<MRData,MRData>> values ) {
+ Hashtable<MRData,Bag> built_table = new Hashtable<MRData,Bag>(Config.map_cache_size);
+ for ( Tuple2<MRData,MRData> t: values ) {
+ Bag entries = built_table.get(t._1);
+ built_table.put(t._1,
+ (entries == null)
+ ? (new Bag(t._2))
+ : entries.add_element(t._2));
+ };
+ return built_table;
+ }
+
+ /** Evaluate MRQL physical operators using Spark
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a Spark RDD
+ */
+ final public JavaRDD<MRData> evalD ( final Tree e, final Environment env ) {
+ try {
+ match e {
+ case MapAggregateReduce(`m,`r,null,_,`s,`o):
+ return evalD(#<MapReduce(`m,`r,`s,`o)>,env);
+ case CrossAggregateProduct(`mx,`my,`r,null,_,`x,`y):
+ return evalD(#<CrossProduct(`mx,`my,`r,`x,`y)>,env);
+ case MapAggregateReduce2(`mx,`my,`r,null,_,`x,`y,`o):
+ return evalD(#<MapReduce2(`mx,`my,`r,`x,`y,`o)>,env);
+ case MapAggregateJoin(`mx,`my,`r,null,_,`x,`y):
+ return evalD(#<MapJoin(`mx,`my,`r,`x,`y)>,env);
+ case cMap(`f,`s):
+ return eval(s,env).flatMap(cmap_fnc(f,env));
+ case MapReduce(`m,`r,`s,`o):
+ return groupBy(eval(s,env).flatMap(cmap_fnc(m,env)),r,env,o);
+ case MapCombineReduce(`m,`c,`r,`s,`o):
+ return groupBy(eval(s,env).flatMap(cmap_fnc(m,env))
+ .mapPartitions(combiner_fnc(evalF(c,env))),r,env,o);
+ case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,`o):
+ return evalD(#<MapReduce2(`mx,`my,`r,`x,`y,`o)>,env).mapPartitions(combiner_fnc(evalF(c,env)));
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ final org.apache.mrql.Function fr = evalF(r,env);
+ return eval(x,env)
+ .flatMap(cmap_fnc(mx,env))
+ .cartesian(eval(y,env).flatMap(cmap_fnc(my,env)))
+ .flatMap(new FlatMapFunction<Tuple2<MRData,MRData>,MRData>() {
+ public Iterable<MRData> call ( Tuple2<MRData,MRData> value ) {
+ return (Bag)fr.eval(new Tuple(value._1,value._2));
+ }
+ });
+ case MapReduce2(`mx,`my,`r,`x,`y,`o):
+ final org.apache.mrql.Function fx = evalF(mx,env);
+ final org.apache.mrql.Function fy = evalF(my,env);
+ final org.apache.mrql.Function fr = evalF(r,env);
+ JavaPairRDD<MRData,MRData> xs
+ = eval(x,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
+ public Iterable<Tuple2<MRData,MRData>> call ( MRData value ) {
+ return joinIterator(((Bag)fx.eval(value)).iterator());
+ }
+ });
+ JavaPairRDD<MRData,MRData> ys
+ = eval(y,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
+ public Iterable<Tuple2<MRData,MRData>> call ( MRData value ) {
+ return joinIterator(((Bag)fy.eval(value)).iterator());
+ }
+ });
+ return xs.cogroup(ys)
+ .flatMap(new FlatMapFunction<Tuple2<MRData,Tuple2<List<MRData>,List<MRData>>>,MRData>() {
+ public Iterable<MRData> call ( Tuple2<MRData,Tuple2<List<MRData>,List<MRData>>> value ) {
+ return (Bag)fr.eval(new Tuple(bag(value._2._1),bag(value._2._2)));
+ }
+ });
+ case GroupByJoin(`kx,`ky,`gx,`gy,`mp,`c,`r,`x,`y,`o):
+ final int n = (int)Math.floor(Math.sqrt(4.0*Config.nodes));
+ final int m = n;
+ if (Config.trace)
+ System.out.println("*** Using a groupBy join on a "+n+"*"+m+" grid of partitions");
+ final org.apache.mrql.Function fkx = evalF(kx,env);
+ final org.apache.mrql.Function fky = evalF(ky,env);
+ final org.apache.mrql.Function fgx = evalF(gx,env);
+ final org.apache.mrql.Function fgy = evalF(gy,env);
+ final org.apache.mrql.Function fm = evalF(mp,env);
+ final org.apache.mrql.Function fc = evalF(c,env);
+ final org.apache.mrql.Function fr = evalF(r,env);
+ final MRData one = new MR_byte(1);
+ final MRData two = new MR_byte(2);
+ final JavaPairRDD<MRData,MRData> xs
+ = eval(x,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
+ public Iterable<Tuple2<MRData,MRData>> call ( final MRData value ) {
+ return new Iterable<Tuple2<MRData,MRData>>() {
+ public Iterator<Tuple2<MRData,MRData>> iterator() {
+ return new Iterator<Tuple2<MRData,MRData>>() {
+ int i = 0;
+ public Tuple2<MRData,MRData> next () {
+ MRData key = new MR_int((fgx.eval(value).hashCode() % m)+m*i);
+ i++;
+ return new Tuple2<MRData,MRData>(key,new Tuple(one,value));
+ }
+ public boolean hasNext () {
+ return i < n;
+ }
+ public void remove () {}
+ };
+ }
+ };
+ }
+ });
+ final JavaPairRDD<MRData,MRData> ys
+ = eval(y,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
+ public Iterable<Tuple2<MRData,MRData>> call ( final MRData value ) {
+ return new Iterable<Tuple2<MRData,MRData>>() {
+ public Iterator<Tuple2<MRData,MRData>> iterator() {
+ return new Iterator<Tuple2<MRData,MRData>>() {
+ int j = 0;
+ public Tuple2<MRData,MRData> next () {
+ MRData key = new MR_int((fgy.eval(value).hashCode() % n)*m+j);
+ j++;
+ return new Tuple2<MRData,MRData>(key,new Tuple(two,value));
+ }
+ public boolean hasNext () {
+ return j < m;
+ }
+ public void remove () {}
+ };
+ }
+ };
+ }
+ });
+ return xs.union(ys).groupByKey(Config.nodes)
+ .mapPartitions(new FlatMapFunction<Iterator<Tuple2<MRData,List<MRData>>>,MRData>() {
+ public Iterable<MRData> call ( final Iterator<Tuple2<MRData,List<MRData>>> value ) {
+ Bag xb = new Bag();
+ Bag yb = new Bag();
+ while (value.hasNext()) {
+ Tuple2<MRData,List<MRData>> t = value.next();
+ for ( MRData e: t._2 ) {
+ Tuple p = (Tuple)e;
+ if (((MR_byte)p.first()).get() == 1)
+ xb.add(new Tuple(t._1,p.second()));
+ else yb.add(new Tuple(t._1,p.second()));
+ }
+ };
+ final Bag b = MapReduceAlgebra.mergeGroupByJoin(fkx,fky,fgx,fgy,fm,fc,fr,xb,yb);
+ xb = null; yb = null;
+ return new Iterable<MRData>() {
+ public Iterator<MRData> iterator() {
+ return b.iterator();
+ }
+ };
+ }
+ });
+ case MapJoin(`mx,`my,`r,`x,`y):
+ final org.apache.mrql.Function fx = evalF(mx,env);
+ final org.apache.mrql.Function fy = evalF(my,env);
+ final org.apache.mrql.Function fr = evalF(r,env);
+ final Broadcast<List<Tuple2<MRData,MRData>>> ys
+ = spark_context.broadcast(eval(y,env).flatMap(new PairFlatMapFunction<MRData,MRData,MRData>() {
+ public Iterable<Tuple2<MRData,MRData>> call ( MRData value ) {
+ return joinIterator(((Bag)fy.eval(value)).iterator());
+ }
+ }).collect());
+ return eval(x,env).flatMap(new FlatMapFunction<MRData,MRData>() {
+ final Hashtable<MRData,Bag> built_table = make_built_table(ys.value());
+ public Iterable<MRData> call ( MRData value ) {
+ final Iterator<MRData> i = ((Bag)fx.eval(value)).iterator();
+ return new Iterable<MRData>() {
+ public Iterator<MRData> iterator() {
+ return new Iterator<MRData>() {
+ Tuple p;
+ Iterator<MRData> ix = null;
+ public MRData next () {
+ return ix.next();
+ }
+ public boolean hasNext () {
+ if (ix != null && ix.hasNext())
+ return true;
+ while (i.hasNext()) {
+ p = (Tuple)i.next();
+ MRData pd = built_table.get(p.first());
+ Bag bb = ((Bag)fr.eval(new Tuple(p.second(),(pd == null) ? empty_bag : pd)));
+ ix = bb.iterator();
+ if (ix.hasNext())
+ return true;
+ };
+ return false;
+ }
+ public void remove () {}
+ };
+ }
+ };
+ }
+ });
+ case BinarySource(`file,_):
+ String path = ((MR_string)evalE(file,env)).get();
+ new BinaryDataSource(path,Plan.conf);
+ return containerData(spark_context.sequenceFile(file.stringValue(),
+ MRContainer.class,MRContainer.class,
+ Config.nodes));
+ case ParsedSource(`parser,`file,...args):
+ String path = ((MR_string)evalE(file,env)).get();
+ Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ new ParsedDataSource(path,p,args,Plan.conf);
+ dump_source_dir();
+ return containerData(spark_context.hadoopFile(path,SparkParsedInputFormat.class,
+ MRContainer.class,MRContainer.class,
+ Config.nodes));
+ case Merge(`x,`y):
+ return eval(x,env).union(eval(y,env));
+ case Repeat(lambda(`v,`b),`s,`n):
+ int max_num = ((MR_int)evalE(n,env)).get();
+ JavaRDD<MRData> rd;
+ JavaRDD<MRData> res = eval(s,env).cache();
+ int i = 0;
+ boolean cont = true;
+ do {
+ rd = eval(b,new Environment(v.toString(),new MR_rdd(res),env)).cache();
+ res = rd.map(get_first).cache();
+ Integer true_results
+ = rd.aggregate(new Integer(0),
+ new Function2<Integer,MRData,Integer>() {
+ public Integer call ( Integer x, MRData y ) {
+ return (((MR_bool)((Tuple)y).second()).get()) ? x+1 : x;
+ }
+ },
+ new Function2<Integer,Integer,Integer>() {
+ public Integer call ( Integer x, Integer y ) { return x+y; }
+ });
+ i++;
+ cont = true_results > 0 && i <= max_num;
+ System.err.println("*** Repeat #"+i+": "+true_results+" true results");
+ } while (cont);
+ return res;
+ case Closure(lambda(`v,`b),`s,`m):
+ int max_num = ((MR_int)evalE(m,env)).get();
+ JavaRDD<MRData> res = eval(s,env).cache();
+ long n = 0;
+ long old = 0;
+ int i = 0;
+ boolean cont = true;
+ do {
+ res = eval(b,new Environment(v.toString(),new MR_rdd(res),env)).cache();
+ old = n;
+ n = res.count();
+ i++;
+ System.err.println("*** Repeat #"+i+": "+(old-n)+" new records");
+ } while (old < n && i <= max_num);
+ return res;
+ case Generator(`min,`max,`size):
+ DataSet ds = Plan.generator(((MR_long)evalE(min,env)).get(),
+ ((MR_long)evalE(max,env)).get(),
+ ((MR_long)evalE(size,env)).get());
+ JavaRDD<MRData> rd = null;
+ for ( DataSource d: ds.source )
+ if (rd == null)
+ rd = containerData(spark_context.hadoopFile(d.path,SparkGeneratorInputFormat.class,
+ MRContainer.class,MRContainer.class,1));
+ else rd = rd.union(containerData(spark_context.hadoopFile(d.path,SparkGeneratorInputFormat.class,
+ MRContainer.class,MRContainer.class,1)));
+ return rd;
+ case let(`v,`u,`body):
+ return eval(body,new Environment(v.toString(),evalE(u,env),env));
+ case Let(`v,`u,`body):
+ return eval(body,new Environment(v.toString(),new MR_rdd(eval(u,env)),env));
+ case If(`c,`x,`y):
+ if (((MR_bool)evalE(c,env)).get())
+ return eval(x,env);
+ else return eval(y,env);
+ case `v:
+ if (!v.is_variable())
+ fail;
+ MRData x = variable_lookup(v.toString(),env);
+ if (x != null)
+ if (x instanceof MR_rdd)
+ return ((MR_rdd)x).rdd();
+ x = variable_lookup(v.toString(),global_env);
+ if (x != null)
+ if (x instanceof MR_rdd)
+ return ((MR_rdd)x).rdd();
+ throw new Error("Variable "+v+" is not bound");
+ };
+ throw new Error("Cannot evaluate the Spark plan: "+e);
+ } catch (Error msg) {
+ if (!Config.trace)
+ throw new Error(msg.getMessage());
+ System.err.println(msg.getMessage());
+ throw new Error("Evaluation error in: "+print_query(e));
+ } catch (Exception ex) {
+ System.err.println(ex.getMessage());
+ ex.printStackTrace();
+ throw new Error("Evaluation error in: "+print_query(e));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/SparkGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/SparkGeneratorInputFormat.java b/spark/src/main/java/org/apache/mrql/SparkGeneratorInputFormat.java
new file mode 100644
index 0000000..8aaca3c
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/SparkGeneratorInputFormat.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.*;
+
+
+/** the FileInputFormat for data generators: it creates HDFS files, where each file contains
+ * an (offset,size) pair that generates the range of values [offset,offset+size] */
+final public class SparkGeneratorInputFormat extends SparkMRQLFileInputFormat {
+ public static class GeneratorRecordReader implements RecordReader<MRContainer,MRContainer> {
+ final long offset;
+ final long size;
+ long index;
+ SequenceFile.Reader reader;
+
+ public GeneratorRecordReader ( FileSplit split,
+ Configuration conf ) throws IOException {
+ Path path = split.getPath();
+ FileSystem fs = path.getFileSystem(conf);
+ reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
+ MRContainer key = new MRContainer();
+ MRContainer value = new MRContainer();
+ reader.next(key,value);
+ offset = ((MR_long)((Tuple)(value.data())).first()).get();
+ size = ((MR_long)((Tuple)(value.data())).second()).get();
+ index = -1;
+ }
+
+ public MRContainer createKey () {
+ return new MRContainer(null);
+ }
+
+ public MRContainer createValue () {
+ return new MRContainer(null);
+ }
+
+ public boolean next ( MRContainer key, MRContainer value ) throws IOException {
+ index++;
+ value.set(new MR_long(offset+index));
+ key.set(new MR_long(index));
+ return index < size;
+ }
+
+ public long getPos () throws IOException { return index; }
+
+ public void close () throws IOException { reader.close(); }
+
+ public float getProgress () throws IOException {
+ return index / (float)size;
+ }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split, JobConf job, Reporter reporter ) throws IOException {
+ return (RecordReader<MRContainer,MRContainer>)
+ new GeneratorRecordReader((FileSplit)split,job);
+ }
+
+ /** Insert all results from the generators stored in path into a Bag.
+ * The Bag is lazily constructed.
+ * @param path the path directory that contains the generator data (offset,size)
+ * @return a Bag that contains all data
+ */
+ public Bag materialize ( final Path path ) throws IOException {
+ Configuration conf = Plan.conf;
+ FileSystem fs = path.getFileSystem(conf);
+ final SequenceFile.Reader reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
+ final MRContainer key = new MRContainer();
+ final MRContainer value = new MRContainer();
+ return new Bag(new BagIterator () {
+ long offset = 0;
+ long size = 0;
+ long i = 0;
+ public boolean hasNext () {
+ if (++i >= offset+size)
+ try {
+ if (!reader.next(key,value))
+ return false;
+ offset = ((MR_long)((Tuple)(value.data())).first()).get();
+ size = ((MR_long)((Tuple)(value.data())).second()).get();
+ i = offset;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from a generator");
+ };
+ return true;
+ }
+ public MRData next () {
+ return new MR_long(i);
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/SparkMRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/SparkMRQLFileInputFormat.java b/spark/src/main/java/org/apache/mrql/SparkMRQLFileInputFormat.java
new file mode 100644
index 0000000..af5ebfa
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/SparkMRQLFileInputFormat.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+
+
+/** A superclass for all MRQL FileInputFormats */
+abstract public class SparkMRQLFileInputFormat extends FileInputFormat<MRContainer,MRContainer> implements MRQLFileInputFormat {
+ public SparkMRQLFileInputFormat () {}
+
+ /** record reader for spark */
+ abstract public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split, JobConf job, Reporter reporter ) throws IOException;
+
+ /** materialize the input file into a memory Bag */
+ public Bag materialize ( final Path file ) throws IOException {
+ final JobConf job = new JobConf(Plan.conf,MRQLFileInputFormat.class);
+ setInputPaths(job,file);
+ final InputSplit[] splits = getSplits(job,1);
+ final Reporter reporter = null;
+ final RecordReader<MRContainer,MRContainer> rd = getRecordReader(splits[0],job,reporter);
+ return new Bag(new BagIterator () {
+ RecordReader<MRContainer,MRContainer> reader = rd;
+ MRContainer key = reader.createKey();
+ MRContainer value = reader.createKey();
+ int i = 0;
+ public boolean hasNext () {
+ try {
+ if (reader.next(key,value))
+ return true;
+ do {
+ if (++i >= splits.length)
+ return false;
+ reader.close();
+ reader = getRecordReader(splits[i],job,reporter);
+ } while (!reader.next(key,value));
+ return true;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from an intermediate result");
+ }
+ }
+ public MRData next () {
+ return value.data();
+ }
+ });
+ }
+
+ /** materialize the entire dataset into a Bag
+ * @param x the DataSet in HDFS to collect values from
+ * @param strip is not used in MapReduce mode
+ * @return the Bag that contains the collected values
+ */
+ public final Bag collect ( final DataSet x, boolean strip ) throws Exception {
+ Bag res = new Bag();
+ for ( DataSource s: x.source )
+ if (s instanceof RDDDataSource)
+ res = res.union(SparkEvaluator.bag(((RDDDataSource)s).rdd));
+ else if (s.to_be_merged)
+ res = res.union(Plan.merge(s));
+ else res = res.union(s.inputFormat.newInstance().materialize(new Path(s.path)));
+ return res;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/spark/src/main/java/org/apache/mrql/SparkParsedInputFormat.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/mrql/SparkParsedInputFormat.java b/spark/src/main/java/org/apache/mrql/SparkParsedInputFormat.java
new file mode 100644
index 0000000..995f69c
--- /dev/null
+++ b/spark/src/main/java/org/apache/mrql/SparkParsedInputFormat.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.Trees;
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** A FileInputFormat for text files (CVS, XML, JSON, ...) */
+final public class SparkParsedInputFormat extends SparkMRQLFileInputFormat {
+ public static class ParsedRecordReader implements RecordReader<MRContainer,MRContainer> {
+ final FSDataInputStream fsin;
+ final long start;
+ final long end;
+ Iterator<MRData> result;
+ Parser parser;
+
+ public ParsedRecordReader ( FileSplit split,
+ Configuration conf,
+ Class<? extends Parser> parser_class,
+ Trees args ) throws IOException {
+ start = split.getStart();
+ end = start + split.getLength();
+ Path file = split.getPath();
+ FileSystem fs = file.getFileSystem(conf);
+ fsin = fs.open(split.getPath());
+ try {
+ parser = parser_class.newInstance();
+ } catch (Exception ex) {
+ throw new Error("Unrecognized parser:"+parser_class);
+ };
+ parser.initialize(args);
+ parser.open(fsin,start,end);
+ result = null;
+ }
+
+ public MRContainer createKey () {
+ return new MRContainer();
+ }
+
+ public MRContainer createValue () {
+ return new MRContainer();
+ }
+
+ public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
+ while (result == null || !result.hasNext()) {
+ String s = parser.slice();
+ if (s == null)
+ return false;
+ result = parser.parse(s).iterator();
+ };
+ value.set((MRData)result.next());
+ key.set(new MR_long(fsin.getPos()));
+ return true;
+ }
+
+ public synchronized long getPos () throws IOException { return fsin.getPos(); }
+
+ public synchronized void close () throws IOException { fsin.close(); }
+
+ public float getProgress () throws IOException {
+ if (end == start)
+ return 0.0f;
+ else return Math.min(1.0f, (getPos() - start) / (float)(end - start));
+ }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split,
+ JobConf job,
+ Reporter reporter ) throws IOException {
+ SparkEvaluator.load_source_dir(); // load the parsed source parameters from a file
+ String path = ((FileSplit)split).getPath().toString();
+ ParsedDataSource ds = (ParsedDataSource)DataSource.get(path,Plan.conf);
+ return new ParsedRecordReader((FileSplit)split,job,ds.parser,(Trees)ds.args);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/BSPBinaryInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/BSPBinaryInputFormat.java b/src/main/java/BSP/BSPBinaryInputFormat.java
deleted file mode 100644
index 8c29a6d..0000000
--- a/src/main/java/BSP/BSPBinaryInputFormat.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.Trees;
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hama.bsp.*;
-import org.apache.hama.HamaConfiguration;
-
-
-/** Input format for hadoop sequence files */
-final public class BSPBinaryInputFormat extends BSPMRQLFileInputFormat {
- public static class BinaryInputRecordReader extends SequenceFileRecordReader<MRContainer,MRContainer> {
- final MRContainer result = new MRContainer();
- final MRData source_num_data;
- final int source_number;
-
- public BinaryInputRecordReader ( FileSplit split,
- BSPJob job,
- int source_number ) throws IOException {
- super(BSPPlan.getConfiguration(job),split);
- this.source_number = source_number;
- source_num_data = new MR_int(source_number);
- }
-
- @Override
- public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
- boolean b = super.next(key,result);
- value.set(new Tuple(source_num_data,result.data()));
- return b;
- }
- }
-
- public RecordReader<MRContainer,MRContainer>
- getRecordReader ( InputSplit split,
- BSPJob job ) throws IOException {
- Configuration conf = BSPPlan.getConfiguration(job);
- String path = ((FileSplit)split).getPath().toString();
- BinaryDataSource ds = (BinaryDataSource)DataSource.get(path,conf);
- return new BinaryInputRecordReader((FileSplit)split,job,ds.source_num);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/BSP/BSPEvaluator.gen
----------------------------------------------------------------------
diff --git a/src/main/java/BSP/BSPEvaluator.gen b/src/main/java/BSP/BSPEvaluator.gen
deleted file mode 100644
index e8fb859..0000000
--- a/src/main/java/BSP/BSPEvaluator.gen
+++ /dev/null
@@ -1,240 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.List;
-import java.io.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hama.HamaConfiguration;
-
-
-/** Evaluate a physical plan in BSP mode using Apache Hama */
-final public class BSPEvaluator extends Evaluator {
-
- /** initialize the BSP evaluator */
- final public void init ( Configuration conf ) {
- Config.bsp_mode = true;
- if (Config.hadoop_mode)
- if (Config.local_mode) {
- conf.set("bsp.master.address","local");
- conf.set("hama.zookeeper.quorum","localhost");
- conf.setInt("bsp.local.tasks.maximum",Config.nodes);
- conf.set("fs.default.name","file:///");
- } else {
- conf.set("bsp.master.address",System.getenv("BSP_MASTER_ADDRESS"));
- conf.set("hama.zookeeper.quorum",System.getenv("HAMA_ZOOKEEPER_QUORUM"));
- conf.setInt("bsp.local.tasks.maximum",Config.nodes);
- conf.set("fs.default.name",System.getenv("FS_DEFAULT_NAME"));
- }
- }
-
- final public void initialize_query () {
- }
-
- /** shutdown the BSP evaluator */
- final public void shutdown ( Configuration conf ) {
- }
-
- final public Configuration new_configuration () {
- return new HamaConfiguration();
- }
-
- final public MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
- if (Config.hadoop_mode)
- return BSPPlan.BSPop.synchronize(peerName,mr_exit);
- else return mr_exit;
- }
-
- final public Bag distribute ( MR_string peerName, Bag s ) {
- if (Config.hadoop_mode)
- return BSPPlan.BSPop.distribute(peerName,s);
- else return s;
- }
-
- /** Evaluate MRQL physical operators using BSP
- * @param plan the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a DataSet (stored in HDFS)
- */
- final public MRData bsp ( Tree plan, Environment env ) throws Exception {
- match plan {
- case BSP(tuple(...ns),`superstep,`init_state,`o,...S):
- int[] os = new int[ns.length()];
- for ( int i = 0; i < os.length; i++ )
- os[i] = (int)((LongLeaf)ns.nth(i)).value();
- DataSet ds = eval(S.head(),env);
- for ( Tree s: S.tail() )
- ds.merge(eval(s,env));
- return BSPPlan.BSP(os,
- closure(superstep,env),
- init_state,
- o.equals(#<true>),
- ds);
- case BSP(`n,`superstep,`init_state,`o,...S):
- DataSet ds = eval(S.head(),env);
- for ( Tree s: S.tail() )
- ds.merge(eval(s,env));
- return BSPPlan.BSP(new int[]{(int)((LongLeaf)n).value()},
- closure(superstep,env),
- init_state,
- o.equals(#<true>),
- ds);
- }
- throw new Error("Cannot perform the BSP plan: "+plan);
- }
-
- /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
- final public Class<? extends MRQLFileInputFormat> parsedInputFormat () {
- return BSPParsedInputFormat.class;
- }
-
- /** return the FileInputFormat for binary files */
- final public Class<? extends MRQLFileInputFormat> binaryInputFormat () {
- return BSPBinaryInputFormat.class;
- }
-
- /** return the FileInputFormat for data generator files */
- final public Class<? extends MRQLFileInputFormat> generatorInputFormat () {
- return BSPGeneratorInputFormat.class;
- }
-
- /** The Aggregate physical operator
- * @param acc_fnc the accumulator function from (T,T) to T
- * @param zero the zero element of type T
- * @param plan the plan that constructs the dataset that contains the bag of values {T}
- * @param env contains bindings fro variables to values (MRData)
- * @return the aggregation result of type T
- */
- final public MRData aggregate ( Tree acc_fnc,
- Tree zero,
- Tree plan,
- Environment env ) throws Exception {
- match plan {
- case BSP(`n,`superstep,`init_state,`o,...S):
- DataSet ds = eval(S.head(),env,"-");
- for ( Tree s: S.tail() )
- ds.merge(eval(s,env,"-"));
- return BSPPlan.BSPaggregate((int)((LongLeaf)n).value(),
- closure(superstep,env),
- init_state,
- closure(acc_fnc,env),
- zero,
- ds);
- }
- throw new Error("Cannot perform the aggregation: "+plan);
- }
-
- final public Tuple loop ( Tree e, Environment env ) {
- throw new Error("The BSP Loop was supposed to be translated to a BSP task");
- }
-
- final public DataSet eval ( final Tree e,
- final Environment env,
- final String counter ) {
- return eval(e,env);
- }
-
- /** Evaluate MRQL BSP physical operators using Hama (returns a DataSet)
- * and print tracing info
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a DataSet (stored in HDFS)
- */
- final public DataSet eval ( final Tree e, final Environment env ) {
- if (Config.trace_execution) {
- tab_count += 3;
- System.out.println(tabs(tab_count)+print_query(e));
- };
- DataSet res = evalD(e,env);
- if (Config.trace_execution)
- try {
- System.out.println(tabs(tab_count)
- +"-> "+res.take(Config.max_bag_size_print));
- tab_count -= 3;
- } catch (Exception ex) {
- throw new Error("Cannot collect the operator output: "+e);
- };
- return res;
- }
-
- /** Evaluate MRQL BSP physical operators using Hama (returns a DataSet)
- * @param e the physical plan
- * @param env contains bindings fro variables to values (MRData)
- * @return a DataSet (stored in HDFS)
- */
- final public DataSet evalD ( final Tree e, final Environment env ) {
- try {
- match e {
- case BSPSource(`n,BinarySource(`file,_)):
- return Plan.binarySource((int)((LongLeaf)n).value(),file.stringValue());
- case BSPSource(`n,ParsedSource(`parser,`file,...args)):
- Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- return Plan.parsedSource((int)((LongLeaf)n).value(),
- p,((MR_string)evalE(file,env)).get(),args);
- case BSPSource(`n,Generator(`min,`max,`size)):
- return Plan.generator((int)((LongLeaf)n).value(),
- ((MR_long)evalE(min,env)).get(),
- ((MR_long)evalE(max,env)).get(),
- ((MR_long)evalE(size,env)).get());
- case Merge(`x,`y):
- return Plan.merge(eval(x,env),eval(y,env));
- case Dump(`s):
- return Plan.fileCache((Bag)evalE(s,env));
- case apply(`f,`arg):
- if (!f.is_variable())
- return ((MR_dataset)evalF(f,env).eval(evalE(arg))).dataset();
- MRData fnc = variable_lookup(f.toString(),global_env);
- if (fnc == null)
- throw new Error("Unknown function: "+f);
- MRData t = evalE(arg,env);
- if (!(t instanceof Tuple))
- throw new Error("Expected a tuple in function application: "+t);
- return ((MR_dataset)((Lambda)fnc).lambda().eval(t)).dataset();
- case `v:
- if (!v.is_variable())
- fail;
- MRData x = variable_lookup(v.toString(),env);
- if (x != null)
- if (x instanceof MR_dataset)
- return ((MR_dataset)x).dataset();
- x = variable_lookup(v.toString(),global_env);
- if (x != null)
- if (x instanceof MR_dataset)
- return ((MR_dataset)x).dataset();
- throw new Error("Variable "+v+" is not bound");
- };
- MRData d = bsp(e,env);
- if (d instanceof MR_dataset)
- return ((MR_dataset)d).dataset();
- throw new Error("Cannot evaluate the BSP plan: "+e);
- } catch (Error msg) {
- if (!Config.trace)
- throw new Error(msg.getMessage());
- System.err.println(msg.getMessage());
- throw new Error("Evaluation error in: "+print_query(e));
- } catch (Exception ex) {
- System.err.println(ex.getMessage());
- ex.printStackTrace();
- throw new Error("Evaluation error in: "+print_query(e));
- }
- }
-}
[11/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MapReduce/MapReducePlan.java
----------------------------------------------------------------------
diff --git a/src/main/java/MapReduce/MapReducePlan.java b/src/main/java/MapReduce/MapReducePlan.java
deleted file mode 100644
index ae243cd..0000000
--- a/src/main/java/MapReduce/MapReducePlan.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-
-
-/** A superclass for Hadoop MapReduce physical operators (files *Operator.java) */
-public class MapReducePlan extends Plan {
-
- MapReducePlan () {}
-
- /** find the number of records in the hadoop MapReduce job output */
- public final static long outputRecords ( Job job ) throws Exception {
- CounterGroup cg = job.getCounters().getGroup("org.apache.hadoop.mapred.Task$Counter");
- long rc = cg.findCounter("REDUCE_OUTPUT_RECORDS").getValue();
- if (rc == 0)
- return cg.findCounter("MAP_OUTPUT_RECORDS").getValue();
- return rc;
- }
-
- /** The Aggregate physical operator
- * @param acc_fnc the accumulator function from (T,T) to T
- * @param zero the zero element of type T
- * @param S the dataset that contains the bag of values {T}
- * @return the aggregation result of type T
- */
- public final static MRData aggregate ( final Tree acc_fnc,
- final Tree zero,
- final DataSet S ) throws Exception {
- MRData res = Interpreter.evalE(zero);
- Function accumulator = functional_argument(Plan.conf,acc_fnc);
- Tuple pair = new Tuple(2);
- for ( DataSource s: S.source )
- if (s.inputFormat != MapReduceBinaryInputFormat.class) {
- pair.set(0,res);
- pair.set(1,aggregate(acc_fnc,zero,
- MapOperation.cMap(Interpreter.identity_mapper,acc_fnc,zero,
- new DataSet(s,0,0),"-")));
- res = accumulator.eval(pair);
- } else {
- Path path = new Path(s.path);
- final FileSystem fs = path.getFileSystem(conf);
- final FileStatus[] ds
- = fs.listStatus(path,
- new PathFilter () {
- public boolean accept ( Path path ) {
- return !path.getName().startsWith("_");
- }
- });
- MRContainer key = new MRContainer(new MR_int(0));
- MRContainer value = new MRContainer(new MR_int(0));
- for ( int i = 0; i < ds.length; i++ ) {
- SequenceFile.Reader reader = new SequenceFile.Reader(fs,ds[i].getPath(),conf);
- while (reader.next(key,value)) {
- pair.set(0,res);
- pair.set(1,value.data());
- res = accumulator.eval(pair);
- };
- reader.close();
- }
- };
- return res;
- }
-
- /** The repeat physical operator. It repeats the loop until all values satisfy
- * the termination condition (when dataset.counter == 0)
- * @param loop the function from DataSet to DataSet to be repeated
- * @param init the initial input DataSet for the loop
- * @param max_num max number of repetitions
- * @return the resulting DataSet from the loop
- */
- public final static DataSet repeat ( Function loop, DataSet init, int max_num ) {
- MR_dataset s = new MR_dataset(init);
- int i = 0;
- do {
- s.dataset = ((MR_dataset)loop.eval(s)).dataset;
- i++;
- System.err.println("*** Repeat #"+i+": "+s.dataset.counter+" true results");
- } while (s.dataset.counter != 0 && i < max_num);
- return s.dataset;
- }
-
- /** The closure physical operator. It repeats the loop until the size
- * of the new DataSet is equal to that of the previous DataSet
- * @param loop the function from DataSet to DataSet to be repeated
- * @param init the initial input DataSet for the loop
- * @param max_num max number of repetitions
- * @return the resulting DataSet from the loop
- */
- public final static DataSet closure ( Function loop, DataSet init, int max_num ) {
- MR_dataset s = new MR_dataset(init);
- int i = 0;
- long n = 0;
- long old = 0;
- do {
- s.dataset = ((MR_dataset)loop.eval(s)).dataset;
- i++;
- System.err.println("*** Repeat #"+i+": "+(s.dataset.records-n)+" new records");
- old = n;
- n = s.dataset.records;
- } while (old < n && i < max_num);
- return s.dataset;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MultipleInputs/DelegatingInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/MultipleInputs/DelegatingInputFormat.java b/src/main/java/MultipleInputs/DelegatingInputFormat.java
deleted file mode 100644
index 0d9b8f9..0000000
--- a/src/main/java/MultipleInputs/DelegatingInputFormat.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapreduce.lib.input;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * An {@link InputFormat} that delegates behavior of paths to multiple other
- * InputFormats.
- *
- * @see MultipleInputs#addInputPath(Job, Path, Class, Class)
- */
-public class DelegatingInputFormat<K, V> extends InputFormat<K, V> {
-
- @SuppressWarnings("unchecked")
- public List<InputSplit> getSplits(JobContext job)
- throws IOException, InterruptedException {
- Configuration conf = job.getConfiguration();
- Job jobCopy =new Job(conf);
- List<InputSplit> splits = new ArrayList<InputSplit>();
- Map<Path, InputFormat> formatMap =
- MultipleInputs.getInputFormatMap(job);
- Map<Path, Class<? extends Mapper>> mapperMap = MultipleInputs
- .getMapperTypeMap(job);
- Map<Class<? extends InputFormat>, List<Path>> formatPaths
- = new HashMap<Class<? extends InputFormat>, List<Path>>();
-
- // First, build a map of InputFormats to Paths
- for (Entry<Path, InputFormat> entry : formatMap.entrySet()) {
- if (!formatPaths.containsKey(entry.getValue().getClass())) {
- formatPaths.put(entry.getValue().getClass(), new LinkedList<Path>());
- }
-
- formatPaths.get(entry.getValue().getClass()).add(entry.getKey());
- }
-
- for (Entry<Class<? extends InputFormat>, List<Path>> formatEntry :
- formatPaths.entrySet()) {
- Class<? extends InputFormat> formatClass = formatEntry.getKey();
- InputFormat format = (InputFormat) ReflectionUtils.newInstance(
- formatClass, conf);
- List<Path> paths = formatEntry.getValue();
-
- Map<Class<? extends Mapper>, List<Path>> mapperPaths
- = new HashMap<Class<? extends Mapper>, List<Path>>();
-
- // Now, for each set of paths that have a common InputFormat, build
- // a map of Mappers to the paths they're used for
- for (Path path : paths) {
- Class<? extends Mapper> mapperClass = mapperMap.get(path);
- if (!mapperPaths.containsKey(mapperClass)) {
- mapperPaths.put(mapperClass, new LinkedList<Path>());
- }
-
- mapperPaths.get(mapperClass).add(path);
- }
-
- // Now each set of paths that has a common InputFormat and Mapper can
- // be added to the same job, and split together.
- for (Entry<Class<? extends Mapper>, List<Path>> mapEntry :
- mapperPaths.entrySet()) {
- paths = mapEntry.getValue();
- Class<? extends Mapper> mapperClass = mapEntry.getKey();
-
- if (mapperClass == null) {
- try {
- mapperClass = job.getMapperClass();
- } catch (ClassNotFoundException e) {
- throw new IOException("Mapper class is not found", e);
- }
- }
-
- FileInputFormat.setInputPaths(jobCopy, paths.toArray(new Path[paths
- .size()]));
-
- // Get splits for each input path and tag with InputFormat
- // and Mapper types by wrapping in a TaggedInputSplit.
- List<InputSplit> pathSplits = format.getSplits(jobCopy);
- for (InputSplit pathSplit : pathSplits) {
- splits.add(new TaggedInputSplit(pathSplit, conf, format.getClass(),
- mapperClass));
- }
- }
- }
-
- return splits;
- }
-
- @Override
- public RecordReader<K, V> createRecordReader(InputSplit split,
- TaskAttemptContext context) throws IOException, InterruptedException {
- return new DelegatingRecordReader<K, V>(split, context);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MultipleInputs/DelegatingMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/MultipleInputs/DelegatingMapper.java b/src/main/java/MultipleInputs/DelegatingMapper.java
deleted file mode 100644
index d72d2b5..0000000
--- a/src/main/java/MultipleInputs/DelegatingMapper.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapreduce.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * An {@link Mapper} that delegates behavior of paths to multiple other
- * mappers.
- *
- * @see MultipleInputs#addInputPath(Job, Path, Class, Class)
- */
-public class DelegatingMapper<K1, V1, K2, V2> extends Mapper<K1, V1, K2, V2> {
-
- private Mapper<K1, V1, K2, V2> mapper;
-
- @SuppressWarnings("unchecked")
- protected void setup(Context context)
- throws IOException, InterruptedException {
- // Find the Mapper from the TaggedInputSplit.
- TaggedInputSplit inputSplit = (TaggedInputSplit) context.getInputSplit();
- mapper = (Mapper<K1, V1, K2, V2>) ReflectionUtils.newInstance(inputSplit
- .getMapperClass(), context.getConfiguration());
-
- }
-
- @SuppressWarnings("unchecked")
- public void run(Context context)
- throws IOException, InterruptedException {
- setup(context);
- mapper.run(context);
- cleanup(context);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MultipleInputs/DelegatingRecordReader.java
----------------------------------------------------------------------
diff --git a/src/main/java/MultipleInputs/DelegatingRecordReader.java b/src/main/java/MultipleInputs/DelegatingRecordReader.java
deleted file mode 100644
index f0d060e..0000000
--- a/src/main/java/MultipleInputs/DelegatingRecordReader.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapreduce.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * This is a delegating RecordReader, which delegates the functionality to the
- * underlying record reader in {@link TaggedInputSplit}
- */
-public class DelegatingRecordReader<K, V> extends RecordReader<K, V> {
- RecordReader<K, V> originalRR;
-
- /**
- * Constructs the DelegatingRecordReader.
- *
- * @param split TaggegInputSplit object
- * @param context TaskAttemptContext object
- *
- * @throws IOException
- * @throws InterruptedException
- */
- @SuppressWarnings("unchecked")
- public DelegatingRecordReader(InputSplit split, TaskAttemptContext context)
- throws IOException, InterruptedException {
- // Find the InputFormat and then the RecordReader from the
- // TaggedInputSplit.
- TaggedInputSplit taggedInputSplit = (TaggedInputSplit) split;
- InputFormat<K, V> inputFormat = (InputFormat<K, V>) ReflectionUtils
- .newInstance(taggedInputSplit.getInputFormatClass(), context
- .getConfiguration());
- originalRR = inputFormat.createRecordReader(taggedInputSplit
- .getInputSplit(), context);
- }
-
- @Override
- public void close() throws IOException {
- originalRR.close();
- }
-
- @Override
- public K getCurrentKey() throws IOException, InterruptedException {
- return originalRR.getCurrentKey();
- }
-
- @Override
- public V getCurrentValue() throws IOException, InterruptedException {
- return originalRR.getCurrentValue();
- }
-
- @Override
- public float getProgress() throws IOException, InterruptedException {
- return originalRR.getProgress();
- }
-
- @Override
- public void initialize(InputSplit split, TaskAttemptContext context)
- throws IOException, InterruptedException {
- originalRR.initialize(((TaggedInputSplit) split).getInputSplit(), context);
- }
-
- @Override
- public boolean nextKeyValue() throws IOException, InterruptedException {
- return originalRR.nextKeyValue();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MultipleInputs/MultipleInputs.java
----------------------------------------------------------------------
diff --git a/src/main/java/MultipleInputs/MultipleInputs.java b/src/main/java/MultipleInputs/MultipleInputs.java
deleted file mode 100644
index 9140512..0000000
--- a/src/main/java/MultipleInputs/MultipleInputs.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapreduce.lib.input;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * This class supports MapReduce jobs that have multiple input paths with
- * a different {@link InputFormat} and {@link Mapper} for each path
- */
-public class MultipleInputs {
- /**
- * Add a {@link Path} with a custom {@link InputFormat} to the list of
- * inputs for the map-reduce job.
- *
- * @param job The {@link Job}
- * @param path {@link Path} to be added to the list of inputs for the job
- * @param inputFormatClass {@link InputFormat} class to use for this path
- */
- @SuppressWarnings("unchecked")
- public static void addInputPath(Job job, Path path,
- Class<? extends InputFormat> inputFormatClass) {
- String inputFormatMapping = path.toString() + ";"
- + inputFormatClass.getName();
- Configuration conf = job.getConfiguration();
- String inputFormats = conf.get("mapred.input.dir.formats");
- conf.set("mapred.input.dir.formats",
- inputFormats == null ? inputFormatMapping : inputFormats + ","
- + inputFormatMapping);
-
- job.setInputFormatClass(DelegatingInputFormat.class);
- }
-
- /**
- * Add a {@link Path} with a custom {@link InputFormat} and
- * {@link Mapper} to the list of inputs for the map-reduce job.
- *
- * @param job The {@link Job}
- * @param path {@link Path} to be added to the list of inputs for the job
- * @param inputFormatClass {@link InputFormat} class to use for this path
- * @param mapperClass {@link Mapper} class to use for this path
- */
- @SuppressWarnings("unchecked")
- public static void addInputPath(Job job, Path path,
- Class<? extends InputFormat> inputFormatClass,
- Class<? extends Mapper> mapperClass) {
-
- addInputPath(job, path, inputFormatClass);
- Configuration conf = job.getConfiguration();
- String mapperMapping = path.toString() + ";" + mapperClass.getName();
- String mappers = conf.get("mapred.input.dir.mappers");
- conf.set("mapred.input.dir.mappers", mappers == null ? mapperMapping
- : mappers + "," + mapperMapping);
-
- job.setMapperClass(DelegatingMapper.class);
- }
-
- /**
- * Retrieves a map of {@link Path}s to the {@link InputFormat} class
- * that should be used for them.
- *
- * @param job The {@link JobContext}
- * @see #addInputPath(JobConf, Path, Class)
- * @return A map of paths to inputformats for the job
- */
- @SuppressWarnings("unchecked")
- static Map<Path, InputFormat> getInputFormatMap(JobContext job) {
- Map<Path, InputFormat> m = new HashMap<Path, InputFormat>();
- Configuration conf = job.getConfiguration();
- String[] pathMappings = conf.get("mapred.input.dir.formats").split(",");
- for (String pathMapping : pathMappings) {
- String[] split = pathMapping.split(";");
- InputFormat inputFormat;
- try {
- inputFormat = (InputFormat) ReflectionUtils.newInstance(conf
- .getClassByName(split[1]), conf);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException(e);
- }
- m.put(new Path(split[0]), inputFormat);
- }
- return m;
- }
-
- /**
- * Retrieves a map of {@link Path}s to the {@link Mapper} class that
- * should be used for them.
- *
- * @param job The {@link JobContext}
- * @see #addInputPath(JobConf, Path, Class, Class)
- * @return A map of paths to mappers for the job
- */
- @SuppressWarnings("unchecked")
- static Map<Path, Class<? extends Mapper>>
- getMapperTypeMap(JobContext job) {
- Configuration conf = job.getConfiguration();
- if (conf.get("mapred.input.dir.mappers") == null) {
- return Collections.emptyMap();
- }
- Map<Path, Class<? extends Mapper>> m =
- new HashMap<Path, Class<? extends Mapper>>();
- String[] pathMappings = conf.get("mapred.input.dir.mappers").split(",");
- for (String pathMapping : pathMappings) {
- String[] split = pathMapping.split(";");
- Class<? extends Mapper> mapClass;
- try {
- mapClass =
- (Class<? extends Mapper>) conf.getClassByName(split[1]);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException(e);
- }
- m.put(new Path(split[0]), mapClass);
- }
- return m;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/MultipleInputs/TaggedInputSplit.java
----------------------------------------------------------------------
diff --git a/src/main/java/MultipleInputs/TaggedInputSplit.java b/src/main/java/MultipleInputs/TaggedInputSplit.java
deleted file mode 100644
index 68bb789..0000000
--- a/src/main/java/MultipleInputs/TaggedInputSplit.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapreduce.lib.input;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * An {@link InputSplit} that tags another InputSplit with extra data for use
- * by {@link DelegatingInputFormat}s and {@link DelegatingMapper}s.
- */
-class TaggedInputSplit extends InputSplit implements Configurable, Writable {
-
- private Class<? extends InputSplit> inputSplitClass;
-
- private InputSplit inputSplit;
-
- @SuppressWarnings("unchecked")
- private Class<? extends InputFormat> inputFormatClass;
-
- @SuppressWarnings("unchecked")
- private Class<? extends Mapper> mapperClass;
-
- private Configuration conf;
-
- public TaggedInputSplit() {
- // Default constructor.
- }
-
- /**
- * Creates a new TaggedInputSplit.
- *
- * @param inputSplit The InputSplit to be tagged
- * @param conf The configuration to use
- * @param inputFormatClass The InputFormat class to use for this job
- * @param mapperClass The Mapper class to use for this job
- */
- @SuppressWarnings("unchecked")
- public TaggedInputSplit(InputSplit inputSplit, Configuration conf,
- Class<? extends InputFormat> inputFormatClass,
- Class<? extends Mapper> mapperClass) {
- this.inputSplitClass = inputSplit.getClass();
- this.inputSplit = inputSplit;
- this.conf = conf;
- this.inputFormatClass = inputFormatClass;
- this.mapperClass = mapperClass;
- }
-
- /**
- * Retrieves the original InputSplit.
- *
- * @return The InputSplit that was tagged
- */
- public InputSplit getInputSplit() {
- return inputSplit;
- }
-
- /**
- * Retrieves the InputFormat class to use for this split.
- *
- * @return The InputFormat class to use
- */
- @SuppressWarnings("unchecked")
- public Class<? extends InputFormat> getInputFormatClass() {
- return inputFormatClass;
- }
-
- /**
- * Retrieves the Mapper class to use for this split.
- *
- * @return The Mapper class to use
- */
- @SuppressWarnings("unchecked")
- public Class<? extends Mapper> getMapperClass() {
- return mapperClass;
- }
-
- public long getLength() throws IOException, InterruptedException {
- return inputSplit.getLength();
- }
-
- public String[] getLocations() throws IOException, InterruptedException {
- return inputSplit.getLocations();
- }
-
- @SuppressWarnings("unchecked")
- public void readFields(DataInput in) throws IOException {
- inputSplitClass = (Class<? extends InputSplit>) readClass(in);
- inputFormatClass = (Class<? extends InputFormat<?, ?>>) readClass(in);
- mapperClass = (Class<? extends Mapper<?, ?, ?, ?>>) readClass(in);
- inputSplit = (InputSplit) ReflectionUtils
- .newInstance(inputSplitClass, conf);
- SerializationFactory factory = new SerializationFactory(conf);
- Deserializer deserializer = factory.getDeserializer(inputSplitClass);
- deserializer.open((DataInputStream)in);
- inputSplit = (InputSplit)deserializer.deserialize(inputSplit);
- }
-
- private Class<?> readClass(DataInput in) throws IOException {
- String className = Text.readString(in);
- try {
- return conf.getClassByName(className);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("readObject can't find class", e);
- }
- }
-
- @SuppressWarnings("unchecked")
- public void write(DataOutput out) throws IOException {
- Text.writeString(out, inputSplitClass.getName());
- Text.writeString(out, inputFormatClass.getName());
- Text.writeString(out, mapperClass.getName());
- SerializationFactory factory = new SerializationFactory(conf);
- Serializer serializer =
- factory.getSerializer(inputSplitClass);
- serializer.open((DataOutputStream)out);
- serializer.serialize(inputSplit);
- }
-
- public Configuration getConf() {
- return conf;
- }
-
- public void setConf(Configuration conf) {
- this.conf = conf;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/AlgebraicOptimization.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/AlgebraicOptimization.gen b/src/main/java/core/AlgebraicOptimization.gen
deleted file mode 100644
index d574e20..0000000
--- a/src/main/java/core/AlgebraicOptimization.gen
+++ /dev/null
@@ -1,315 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-
-/**
- * Optimize MRQL algebra expressions using normalization rules and heuristic optimizations
- */
-public class AlgebraicOptimization extends Simplification {
-
- /** Is this a group-by operation?*/
- private static boolean is_groupBy ( Tree e ) {
- match e {
- case groupBy(...): return true;
- case orderBy(...): return true;
- };
- return false;
- }
-
- /**
- * algebraic optimization (algebra to algebra)
- * @param e the algebraic form to be optimized
- * @return the optimized form
- */
- public static Tree translate ( Tree e ) {
- match e {
- // if the group-by key is the same as the join key, fuse the group-by into the join
- case mapReduce2(lambda(`v,bag(tuple(nth(`v1,0),`mx))),`my,
- lambda(`rv,`rb),
- `groupBy(cmap(lambda(`w,`m),`X)),`Y,`o):
- if (!v1.equals(v) || !Config.groupJoinOpt || ! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- Tree nv = new_var();
- Tree nr = subst(#<nth(`rv,0)>,#<cmap(lambda(`v,bag(`mx)),groupBy(nth(`rv,0)))>,rb);
- Tree res = #<mapReduce2(lambda(`w,cmap(lambda(`nv,bag(tuple(nth(`nv,0),`nv))),`m)),
- `my,lambda(`rv,`nr),`X,`Y,`o)>;
- res = simplify_all(rename(res));
- TypeInference.type_inference(res);
- return translate(res);
- // same for the right join input
- case mapReduce2(`mx,lambda(`v,bag(tuple(nth(`v1,0),`my))),
- lambda(`rv,`rb),
- `X,`groupBy(cmap(lambda(`w,`m),`Y)),`o):
- if (!v1.equals(v) || !Config.groupJoinOpt || ! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- Tree nv = new_var();
- Tree nr = subst(#<nth(`rv,1)>,#<cmap(lambda(`v,bag(`my)),groupBy(nth(`rv,1)))>,rb);
- Tree res = #<mapReduce2(`mx,lambda(`w,cmap(lambda(`nv,bag(tuple(nth(`nv,0),`nv))),`m)),
- lambda(`rv,`nr),`X,`Y,`o)>;
- res = simplify_all(rename(res));
- TypeInference.type_inference(res);
- return translate(res);
- // convert self-joins to single-source mapreduce; Hadoop doesn't like self-joins anyway
- case mapReduce2(lambda(`vx,`bx),lambda(`vy,`by),lambda(`s,`f),`X,`Y,`o):
- if (!alpha_equivalent(X,Y) || !Config.selfJoinOpt)
- fail;
- Tree ny = subst(vy,vx,by);
- Tree tx = null;
- Tree ty = null;
- match TypeInference.type_inference(bx) {
- case _(tuple(_,`t)): tx = t;
- };
- match TypeInference.type_inference(by) {
- case _(tuple(_,`t)): ty = t;
- };
- Tree b = subst(s,#<tuple(cmap(lambda(tuple(n,v1,v2),
- if(call(eq,n,1),bag(v1),bag())),s),
- cmap(lambda(tuple(n,v1,v2),
- if(call(eq,n,2),bag(v2),bag())),s))>,
- f);
- Tree res = #<mapReduce(lambda(`vx,call(plus,
- cmap(lambda(x,bag(tuple(nth(x,0),tuple(1,nth(x,1),typed(null,`ty))))),`bx),
- cmap(lambda(y,bag(tuple(nth(y,0),tuple(2,typed(null,`tx),nth(y,1))))),`ny))),
- lambda(tuple(k,s),`b),
- `X,`o)>;
- res = simplify_all(rename(res));
- TypeInference.type_inference(res);
- return translate(res);
- case mapReduce2(`mx,`my,`r,cmap(lambda(`v,`b),`X),`Y,`o):
- match X {
- case groupBy(...): ;
- case _: return translate(#<mapReduce2(lambda(`v,cmap(`mx,`b)),`my,`r,`X,`Y,`o)>);
- };
- fail
- case mapReduce2(`mx,`my,`r,`X,cmap(lambda(`v,`b),`Y),`o):
- match Y {
- case groupBy(...): ;
- case _: return translate(#<mapReduce2(`mx,lambda(`v,cmap(`my,`b)),`r,`X,`Y,`o)>);
- };
- fail
- case crossProduct(`mx,`my,`r,cmap(lambda(`vx,`bx),`X),cmap(lambda(`vy,`by),`Y)):
- return translate(#<crossProduct(lambda(`vx,cmap(`mx,`bx)),
- lambda(`vy,cmap(`my,`by)),`r,`X,`Y)>);
- case crossProduct(`mx,`my,`r,cmap(lambda(`v,`b),`X),`Y):
- return translate(#<crossProduct(lambda(`v,cmap(`mx,`b)),`my,`r,`X,`Y)>);
- case crossProduct(`mx,`my,`r,`X,cmap(lambda(`v,`b),`Y)):
- return translate(#<crossProduct(`mx,lambda(`v,cmap(`my,`b)),`r,`X,`Y)>);
- case cmap(`m,crossProduct(`mx,`my,lambda(`v,`b),`X,`Y)):
- return translate(#<crossProduct(`mx,`my,lambda(`v,cmap(`m,`b)),`X,`Y)>);
- case cmap(`r,`groupBy1(cmap(`m,`groupBy2(`s)))):
- if (! #[groupBy,orderBy].member(#<`groupBy1>)
- && ! #[groupBy,orderBy].member(#<`groupBy2>))
- fail;
- return #<mapReduce(`(identity()),
- `(translate(r)),
- `(translate(#<cmap(`m,`groupBy2(`s))>)),
- `((#<`groupBy1>.equals(#<orderBy>)) ? #<true> : #<false>))>;
- case cmap(`r,`groupBy(cmap(`m,`s))):
- if (! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- return #<mapReduce(`(translate(m)),
- `(translate(r)),
- `(translate(s)),
- `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
- case `groupBy(cmap(`m,groupBy(`s))):
- if (! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- return #<mapReduce(`(identity()),
- `(identity()),
- `(translate(#<cmap(`m,groupBy(`s))>)),
- `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
- case `groupBy(cmap(`m,`s)):
- if (! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- return #<mapReduce(`(translate(m)),
- `(identity()),
- `(translate(s)),
- `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
- case cmap(`r,`groupBy(`s)):
- if (! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- return #<mapReduce(`(identity()),
- `(translate(r)),
- `(translate(s)),
- `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
- case `groupBy(`s):
- if (! #[groupBy,orderBy].member(#<`groupBy>))
- fail;
- return #<mapReduce(`(identity()),
- `(identity()),
- `(translate(s)),
- `((#<`groupBy>.equals(#<orderBy>)) ? #<true> : #<false>))>;
- case cmap(`m,`s):
- return #<cmap(`(translate(m)),
- `(translate(s)))>;
- // convert self-joins to single-source mapreduce; Hadoop doesn't like self-joins anyway
- case join(lambda(`vx,`bx),lambda(`vy,`by),lambda(`s,`f),`x,`y):
- if (!x.equals(y) || !Config.selfJoinOpt)
- fail;
- Tree ny = subst(vy,vx,by);
- Tree b = subst(s,
- #<tuple(cmap(lambda(tuple(n,v),
- if(call(eq,n,1),bag(v),bag())),s),
- cmap(lambda(tuple(n,v),
- if(call(eq,n,2),bag(v),bag())),s))>,
- f);
- Tree res = #<mapReduce(lambda(`vx,bag(tuple(`bx,tuple(1,`vx)),
- tuple(`ny,tuple(2,`vx)))),
- lambda(tuple(k,s),`b),`x,false)>;
- res = simplify_all(rename(res));
- TypeInference.type_inference(res);
- return translate(res);
- case join(lambda(`vx,`bx),lambda(`vy,`by),`f,`x,`y):
- return translate(#<mapReduce2(lambda(`vx,bag(tuple(`bx,`vx))),
- lambda(`vy,bag(tuple(`by,`vy))),
- `f,`x,`y,false)>);
- case nth(`x,`n):
- match TypeInference.type_inference2(x) {
- case `S(tuple(...bl)):
- if (!is_collection(S))
- fail;
- Tree nv = new_var();
- type_env.insert(nv.toString(),bl.nth((int)n.longValue()));
- return translate(#<cmap(lambda(`nv,`S(nth(`nv,`n))),`x)>);
- };
- fail
- case project(`x,`a):
- match TypeInference.type_inference2(x) {
- case `S(record(...bl)):
- if (!is_collection(S))
- fail;
- for ( Tree b: bl )
- match b {
- case bind(`c,_):
- if (!a.equals(c))
- fail;
- Tree nv = new_var();
- type_env.insert(nv.toString(),#<record(...bl)>);
- return translate(#<cmap(lambda(`nv,`S(project(`nv,`a))),`x)>);
- };
- };
- fail
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(translate(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- /** apply algebraic optimizations multiple times until no change */
- public static Tree translate_all ( Tree e ) {
- Tree ne = translate(e);
- if (e.equals(ne))
- return e;
- else return translate_all(ne);
- }
-
- /**
- * does a form contain a bulk plan that doesn't refer to a given variable?
- * @param e the form
- * @param v the given variable
- * @return true if e contains a bulk plan that doesn't refer to v
- */
- private static boolean contains_plan ( Tree e, Tree v ) {
- match e {
- case lambda(`x,`u): return false;
- case let(...): return false;
- case Let(...): return false;
- case `f(...as):
- if (plan_names.member(#<`f>) && !free_variables(e,#[]).member(v))
- return true;
- for (Tree a: as)
- if (contains_plan(a,v))
- return true;
- return false;
- };
- return false;
- }
-
- /**
- * extract the common factors (common sub-expressions) from a form
- * that do not refer to a given variable
- * @param e the form
- * @param v the given variable
- * @return the list of common factors
- */
- private static Trees common_factors ( Tree e, Tree v ) {
- match e {
- case lambda(`x,`u): return #[];
- case let(...): return #[];
- case Let(...): return #[];
- case `f(...as):
- if (!contains_plan(e,v))
- fail;
- if (plan_names.member(#<`f>) && !free_variables(e,#[]).member(v))
- return #[`e];
- Trees bs = #[];
- for ( Tree a: as )
- bs = bs.append(common_factors(a,v));
- return bs;
- };
- return #[];
- }
-
- /**
- * if a term is used multiple times in a query, factor it out using let-expressions
- * @param e the expression to be factored-out
- * @return the factored-out expression
- */
- public static Tree common_factoring ( Tree e ) {
- match e {
- case `f(...as):
- if (!plan_names.member(#<`f>))
- fail;
- Trees bs = #[];
- Trees binds = #[];
- for ( Tree a: as )
- match a {
- case lambda(`v,`u):
- if (!contains_plan(u,v))
- fail;
- Trees gs = common_factors(u,v);
- Tree nb = u;
- for ( Tree g: gs) {
- Tree nv = new_var();
- nb = subst(g,nv,nb);
- binds = binds.append(#<bind(`nv,`g)>);
- };
- bs = bs.append(#<lambda(`v,`(common_factoring(nb)))>);
- case _: bs = bs.append(common_factoring(a));
- };
- Tree res = #<`f(...bs)>;
- for ( Tree bind: binds )
- match bind {
- case bind(`v,`x):
- res = #<let(`v,`x,`res)>;
- };
- return res;
- case `f(...as):
- Trees bs = #[];
- for ( Tree a: as )
- bs = bs.append(common_factoring(a));
- return #<`f(...bs)>;
- };
- return e;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/BSPTranslator.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/BSPTranslator.gen b/src/main/java/core/BSPTranslator.gen
deleted file mode 100644
index a2af7a6..0000000
--- a/src/main/java/core/BSPTranslator.gen
+++ /dev/null
@@ -1,753 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-
-
-/** Translates physical plans to BSP plans to be executed on Hama */
-public class BSPTranslator extends TypeInference {
- final static int orM = ClassImporter.find_method_number("or",#[bool,bool]);
- final static int andM = ClassImporter.find_method_number("and",#[bool,bool]);
- final static int notM = ClassImporter.find_method_number("not",#[bool]);
- final static int eqM = ClassImporter.find_method_number("eq",#[int,int]);
- final static int plusM = ClassImporter.find_method_number("plus",#[int,int]);
- final static int neqM = ClassImporter.find_method_number("neq",#[int,int]);
- final static int geqM = ClassImporter.find_method_number("geq",#[long,long]);
- final static int gtM = ClassImporter.find_method_number("gt",#[int,int]);
- final static int unionM = ClassImporter.find_method_number("union",#[bag(any),bag(any)]);
- final static int countM = ClassImporter.find_method_number("count",#[bag(any)]);
- final static int coerceM = ClassImporter.find_method_number("coerce",#[any,int]);
-
- final static Trees planNames = plans_with_distributed_lambdas
- .append(#[ParsedSource,Generator,BinarySource,GroupByJoin,Repeat,Closure,repeat]);
-
- private static int source_num = 0;
-
- public static void reset () {
- source_num = 0;
- }
-
- private static Tree new_source ( Tree e ) {
- match e {
- case `f(...):
- if (planNames.member(#<`f>))
- return preprocess(e);
- };
- return #<BSPSource(`(source_num++),`(preprocess(e)))>;
- }
-
- /** add a new source num to every BSP operation */
- private static Tree preprocess ( Tree e ) {
- match e {
- case lambda(`v,`b):
- return #<lambda(`v,`(preprocess(b)))>;
- case Aggregate(`acc,`zero,`S):
- return #<Aggregate(`acc,`zero,
- `(new_source(S)))>;
- case cMap(`m,`S):
- return #<cMap(`m,
- `(new_source(S)))>;
- case AggregateMap(`m,`acc,`zero,`S):
- return #<AggregateMap(`m,`acc,`zero,
- `(new_source(S)))>;
- case MapReduce(`m,`r,`S,`o):
- return #<MapReduce(`m,`r,
- `(new_source(S)),`o)>;
- case MapAggregateReduce(`m,`r,`acc,`zero,`S,`o):
- return #<MapAggregateReduce(`m,`r,`acc,`zero,
- `(new_source(S)),`o)>;
- case MapCombineReduce(`m,`c,`r,`S,`o):
- return #<MapCombineReduce(`m,`c,`r,
- `(new_source(S)),`o)>;
- case MapReduce2(`mx,`my,`r,`x,`y,`o):
- return #<MapReduce2(`mx,`my,`r,
- `(new_source(x)),
- `(new_source(y)),`o)>;
- case MapReduce2(`mx,`my,`c,`r,`x,`y,`o):
- return #<MapCombineReduce2(`mx,`my,`c,`r,
- `(new_source(x)),
- `(new_source(y)),`o)>;
- case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,`o):
- return #<MapAggregateReduce2(`mx,`my,`r,`acc,`zero,
- `(new_source(x)),
- `(new_source(y)),`o)>;
- case MapJoin(`mx,`my,`r,`x,`y):
- return #<MapJoin(`mx,`my,`r,
- `(new_source(x)),
- `(new_source(y)))>;
- case MapAggregateJoin(`mx,`my,`r,`acc,`zero,`x,`y):
- return #<MapAggregateJoin(`mx,`my,`r,`acc,`zero,
- `(new_source(x)),
- `(new_source(y)))>;
- case CrossProduct(`mx,`my,`r,`x,`y):
- return #<CrossProduct(`mx,`my,`r,
- `(new_source(x)),
- `(new_source(y)))>;
- case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
- return #<CrossAggregateProduct(`mx,`my,`r,`acc,`zero,
- `(new_source(x)),
- `(new_source(y)))>;
- case GroupByJoin(`kx,`ky,`gx,`gy,`mr,`c,`r,`x,`y,`o):
- return #<GroupByJoin(`kx,`ky,`gx,`gy,`mr,`c,`r,
- `(new_source(x)),
- `(new_source(y)),`o)>;
- case Repeat(`f,`ds,`max):
- return #<Repeat(`(preprocess(f)),`(preprocess(ds)),`max)>;
- case repeat(`f,`ds,`max):
- return #<repeat(`(preprocess(f)),`(preprocess(ds)),`max)>;
- case Closure(`f,`ds,`max):
- return #<Closure(`(preprocess(f)),`(preprocess(ds)),`max)>;
- case `f(...):
- if (! #[ParsedSource,Generator,BinarySource].member(#<`f>))
- fail;
- return #<BSPSource(`(source_num++),`e)>;
- case `f(...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(preprocess(a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- /** returns the source num of a BSP operation */
- private static int source_num ( Tree e ) {
- match e {
- case BSP(`i,...):
- return (int)((LongLeaf)i).value();
- case BSPSource(`i,_):
- return (int)((LongLeaf)i).value();
- };
- return -1;
- }
-
- private static Tree subst_getCache_num ( int n, int m, Tree e ) {
- match e {
- case getCache(`cache,`k):
- if (!k.equals(#<`n>))
- fail;
- return #<getCache(`cache,`m)>;
- case `f(...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(subst_getCache_num(n,m,a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- private static Tree subst_setCache_num ( int n, int m, Tree e ) {
- match e {
- case setCache(`cache,`k,...r):
- if (!k.equals(#<`n>))
- fail;
- return #<setCache(`cache,`m,...r)>;
- case `f(...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(subst_setCache_num(n,m,a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- private static Tree set_cache_num ( Tree e, int n ) {
- match e {
- case BSP(`m,...r):
- return subst_setCache_num((int)((LongLeaf)m).value(),n,#<BSP(`n,...r)>);
- case BSPSource(_,`x):
- return #<BSPSource(`n,`x)>;
- };
- return e;
- }
-
- // return the source numbers of a BSP operation
- private static Trees source_nums ( Tree e ) {
- match e {
- case BSP(`i,...):
- return #[`i];
- case Merge(`x,`y):
- return source_nums(x).append(source_nums(y));
- case BSPSource(`i,_):
- return #[`i];
- };
- return #[];
- }
-
- private static Trees source_nums ( Trees s ) {
- Trees res = #[];
- for ( Tree x: s )
- res = res.append(source_nums(x));
- return res;
- }
-
- private static Tree getCache ( Tree var, Tree e, Tree body ) {
- Trees ns = source_nums(e);
- if (ns.length() > 0)
- return subst(var,#<getCache(cache,...ns)>,body);
- return body;
- }
-
- private static Tree getCache ( Tree var, Trees s, Tree body ) {
- Trees ns = source_nums(s);
- return subst(var,#<getCache(cache,...ns)>,body);
- }
-
- /** optimize a BSP plan after BSP fusion */
- private static Tree post_simplify ( Tree e ) {
- match e {
- case cmap(lambda(`x,`S(`a)),`b):
- if (is_collection(S))
- return post_simplify(#<map(lambda(`x,`a),`b)>);
- else fail
- case `f(...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(post_simplify(a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- private static Tree processBSP ( Tree e ) {
- return post_simplify(Simplification.simplify_all(Simplification.rename(e)));
- }
-
- private static Tree mkBSP ( Tree superstep, Tree state, Tree orderp, Tree input ) {
- int rn = source_num++;
- superstep = processBSP(subst(#<o_>,#<`rn>,getCache(#<i_>,input,superstep)));
- return #<BSP(`rn,`superstep,`state,`orderp,`input)>;
- }
-
- private static Tree mkBSPL ( Tree superstep, Tree state, Tree orderp, Trees input ) {
- int rn = source_num++;
- superstep = processBSP(subst(#<o_>,#<`rn>,getCache(#<i>,input,superstep)));
- return #<BSP(`rn,`superstep,`state,`orderp,...input)>;
- }
-
- private static Tree mkBSPL ( int[] ns, Tree superstep, Tree state, Tree orderp, Trees input ) {
- superstep = processBSP(subst(#<o_>,#<`(ns[0])>,getCache(#<i_>,input,superstep)));
- Trees s = #[];
- for ( int n: ns )
- s = s.append(#<`n>);
- return #<BSP(tuple(...s),`superstep,`state,`orderp,...input)>;
- }
-
- private static Tree mkBSPL ( int n, Tree superstep, Tree state, Tree orderp, Trees input ) {
- superstep = processBSP(subst(#<o_>,#<`n>,getCache(#<i_>,input,superstep)));
- return #<BSP(`n,`superstep,`state,`orderp,...input)>;
- }
-
- private static Tree mkBSP2 ( Tree superstep, Tree state, Tree orderp, Tree left, Tree right ) {
- int rn = source_num++;
- superstep = processBSP(subst(#<o_>,#<`rn>,getCache(#<i_>,left,
- getCache(#<j_>,right,superstep))));
- return #<BSP(`rn,`superstep,`state,`orderp,`left,`right)>;
- }
-
- /** construct a BSP plan from a physical plan
- * @param e the physical plan
- * @return the BSP plan
- */
- private static Tree mr2bsp ( Tree e ) {
- match e {
- case Aggregate(`acc,`zero,`S):
- return #<Aggregate(`acc,`zero,`(mr2bsp(S)))>;
- case cMap(`m,`S):
- return mkBSP(#<lambda(tuple(cache,ms,k,peer),
- setCache(cache,o_,cmap(`m,i_),
- tuple(BAG(),tuple(),TRUE())))>,
- #<tuple()>,
- #<false>,
- mr2bsp(S));
- case AggregateMap(`m,`acc,`zero,`S):
- return mkBSP(#<lambda(tuple(cache,ms,k,peer),
- setCache(cache,o_,aggregate(`acc,`zero,cmap(`m,i_)),
- tuple(BAG(),tuple(),TRUE())))>,
- #<tuple()>,
- #<false>,
- mr2bsp(S));
- case MapReduce(`m,`r,`S,`o):
- return mkBSP(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(cmap(lambda(tuple(k,c),
- bag(tuple(k,tuple(k,c)))),
- cmap(`m,i_)),
- false,
- FALSE()),
- setCache(cache,o_,`(o.equals(#<true>) // need to sort the result?
- ? #<cmap(lambda(tuple(k,s),
- cmap(lambda(x,bag(tuple(x,k))),
- apply(`r,tuple(k,s)))),
- groupBy(ms))>
- : #<cmap(`r,groupBy(ms))>),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- o,
- mr2bsp(S));
- case MapAggregateReduce(`m,`r,`acc,`zero,`S,_):
- return mkBSP(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(cmap(lambda(tuple(k,c),
- bag(tuple(k,tuple(k,c)))),
- cmap(`m,i_)),
- false,
- FALSE()),
- setCache(cache,o_,aggregate(`acc,`zero,cmap(`r,groupBy(ms))),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- #<false>,
- mr2bsp(S));
- case MapCombineReduce(`m,`c,`r,`S,`o):
- return mkBSP(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(cmap(lambda(tuple(k,s),
- cmap(lambda(x,bag(tuple(k,tuple(k,x)))),
- apply(`c,tuple(k,s)))),
- groupBy(cmap(`m,i_))),
- false,
- FALSE()),
- setCache(cache,o_,`(o.equals(#<true>) // need to sort the result?
- ? #<cmap(lambda(tuple(k,s),
- cmap(lambda(x,bag(tuple(x,k))),
- apply(`r,tuple(k,s)))),
- groupBy(ms))>
- : #<cmap(`r,groupBy(ms))>),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- o,
- mr2bsp(S));
- case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,`o):
- return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(callM(union,`unionM,
- cmap(lambda(tuple(kx,x),
- bag(tuple(kx,tuple(kx,tuple(1,x))))),
- cmap(`mx,i_)),
- cmap(lambda(tuple(ky,y),
- bag(tuple(ky,tuple(ky,tuple(2,y))))),
- cmap(`my,j_))),
- false,
- FALSE()),
- setCache(cache,o_,cmap(lambda(tuple(k,s),
- cmap(lambda(tuple(kk,ss),
- cmap(lambda(x,bag(tuple(kk,x))),
- apply(`c,tuple(kk,ss)))),
- groupBy(apply(`r,
- tuple(cmap(lambda(tuple(kx,x),
- if(callM(eq,`eqM,kx,1),
- bag(x),
- bag())),
- s),
- cmap(lambda(tuple(ky,y),
- if(callM(eq,`eqM,ky,2),
- bag(y),
- bag())),
- s)))))),
- groupBy(ms)),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- o,
- mr2bsp(x),
- mr2bsp(y));
- case MapReduce2(`mx,`my,`r,`x,`y,`o):
- return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(callM(union,`unionM,
- cmap(lambda(tuple(kx,x),
- bag(tuple(kx,tuple(kx,tuple(1,x))))),
- cmap(`mx,i_)),
- cmap(lambda(tuple(ky,y),
- bag(tuple(ky,tuple(ky,tuple(2,y))))),
- cmap(`my,j_))),
- false,
- FALSE()),
- setCache(cache,o_,cmap(lambda(tuple(k,s),
- cmap(lambda(x,bag(`(o.equals(#<true>) ? #<tuple(x,k)> : #<x>))),
- apply(`r,
- tuple(cmap(lambda(tuple(kx,x),
- if(callM(eq,`eqM,kx,1),
- bag(x),
- bag())),
- s),
- cmap(lambda(tuple(ky,y),
- if(callM(eq,`eqM,ky,2),
- bag(y),
- bag())),
- s))))),
- groupBy(ms)),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- o,
- mr2bsp(x),
- mr2bsp(y));
- case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,_):
- return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(callM(union,`unionM,
- cmap(lambda(tuple(kx,x),
- bag(tuple(kx,tuple(kx,tuple(1,x))))),
- cmap(`mx,i_)),
- cmap(lambda(tuple(ky,y),
- bag(tuple(ky,tuple(ky,tuple(2,y))))),
- cmap(`my,j_))),
- false,
- FALSE()),
- setCache(cache,o_,aggregate(`acc,`zero,
- cmap(lambda(tuple(k,s),
- apply(`r,tuple(cmap(lambda(tuple(kx,x),
- if(callM(eq,`eqM,kx,1),
- bag(x),
- bag())),
- s),
- cmap(lambda(tuple(ky,y),
- if(callM(eq,`eqM,ky,2),
- bag(y),
- bag())),
- s)))),
- groupBy(ms))),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- #<false>,
- mr2bsp(x),
- mr2bsp(y));
- case MapJoin(`mx,`my,`r,`x,`y):
- return mr2bsp(#<MapReduce2(`mx,`my,
- lambda(tuple(xs,ys),cmap(lambda(x,apply(`r,tuple(x,ys))),xs)),
- `x,`y,false)>);
- case MapAggregateJoin(`mx,`my,`r,`acc,`zero,`x,`y):
- return mr2bsp(#<MapAggregateReduce2(`mx,`my,
- lambda(tuple(xs,ys),cmap(lambda(x,apply(`r,tuple(x,ys))),xs)),
- `acc,`zero,`x,`y)>);
- case GroupByJoin(`kx,`ky,lambda(`vx,`gx),lambda(`vy,`gy),`mr,`c,`r,`x,`y,`o):
- int n = 8;
- int m = 8;
- System.err.println("*** Using a groupBy join on a "+n+"*"+m+" grid of partitions");
- Tree xkey = #<cmap(lambda(i,bag(tuple(call(plus,call(mod,call(hash_code,`gx),`m),call(times,`m,i)),
- tuple(1,call(plus,call(mod,call(hash_code,`gx),`m),call(times,`m,i)),`vx)))),
- range(0,`(n-1)))>;
- Tree ykey = #<cmap(lambda(j,bag(tuple(call(plus,call(times,call(mod,call(hash_code,`gy),`n),`m),j),
- tuple(2,call(plus,call(times,call(mod,call(hash_code,`gy),`n),`m),j),`vy)))),
- range(0,`(m-1)))>;
- type_inference(xkey);
- type_inference(ykey);
- xkey = PlanGeneration.makePlan(xkey);
- ykey = PlanGeneration.makePlan(ykey);
- return mkBSP2(#<lambda(tuple(cache,ms,map_step,peer),
- if(map_step,
- tuple(callM(union,`unionM,
- cmap(lambda(`vx,`xkey),i_),
- cmap(lambda(`vy,`ykey),j_)),
- false,
- FALSE()),
- setCache(cache,o_,
- mergeGroupByJoin(`kx,`ky,lambda(`vx,`gx),lambda(`vy,`gy),`mr,`c,`r,
- cmap(lambda(tuple(kx,p,x),
- if(callM(eq,`eqM,kx,1),bag(tuple(p,x)),bag())),ms),
- cmap(lambda(tuple(ky,p,y),
- if(callM(eq,`eqM,ky,2),bag(tuple(p,y)),bag())),ms),`o),
- tuple(BAG(),false,TRUE()))))>,
- #<true>,
- o,
- mr2bsp(x),
- mr2bsp(y));
- case CrossProduct(`mx,`my,`r,`x,`y):
- return mkBSP(#<lambda(tuple(cache,ms,ys,peer),
- tuple(BAG(),
- setCache(cache,o_,
- cmap(lambda(x,
- cmap(lambda(y,apply(`r,tuple(x,y))),
- cmap(`my,ys))),
- cmap(`mx,i_)),
- tuple()),
- TRUE()))>,
- #<Collect(`(mr2bsp(y)))>,
- #<false>,
- mr2bsp(x));
- case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
- return mkBSP(#<lambda(tuple(cache,ms,ys,peer),
- tuple(BAG(),
- setCache(cache,o_,
- aggregate(`acc,`zero,
- cmap(lambda(x,
- cmap(lambda(y,apply(`r,tuple(x,y))),
- cmap(`my,ys))),
- cmap(`mx,i_)),
- tuple())),
- TRUE()))>,
- #<Collect(`(mr2bsp(y)))>,
- #<false>,
- mr2bsp(x));
- case Repeat(lambda(`v,`b),`ds,`max):
- Tree step = bspSimplify(mr2bsp(b));
- int step_cache_num = source_num(step);
- match step {
- case BSP(`n,`s,`k0,_,...as):
- Tree ds_source = mr2bsp(ds);
- int ds_cache_num = source_num(ds_source);
- ds_source = set_cache_num(ds_source,step_cache_num);
- // the initial values of all data sources
- Trees sources = #[`ds_source];
- Tree step_input = #<0>;
- for ( Tree x: as )
- match x {
- case BSPSource(`j,`y):
- if (y.equals(v))
- step_input = j;
- else sources = sources.append(x);
- case _: sources = sources.append(x);
- };
- s = subst_getCache_num((int)((LongLeaf)step_input).value(),step_cache_num,s);
- return mkBSPL(step_cache_num,
- #<lambda(tuple(cache,ms,tuple(k,steps),peer),
- let(tuple(ts,kk,step_end),
- apply(`s,tuple(cache,ms,k,peer)),
- if(step_end, // end of repeat step
- setCache(cache,`step_cache_num,
- map(lambda(tuple(x,bb),x),
- getCache(cache,`step_cache_num)),
- tuple(bag(),
- tuple(`k0,callM(plus,`plusM,steps,1)),
- if(callM(gt,`gtM,steps,`max),
- TRUE(), // if # of steps <= limit, exit
- callM(not,`notM, // ... else check the stopping condition
- aggregate(lambda(tuple(x,y),callM(or,`orM,x,y)),
- false,
- map(lambda(tuple(x,bb),bb),
- getCache(cache,`step_cache_num))))))),
- tuple(ts,tuple(kk,steps),FALSE()))))>,
- #<tuple(`k0,1)>,
- #<false>,
- sources);
- case `x: throw new Error("Cannot compile the repeat function: "+x);
- }
- // when the repeat variable is in memory
- case repeat(lambda(`v,`b),`ds,`max):
- Tree step = bspSimplify(mr2bsp(b));
- int step_cache_num = source_num(step);
- step = subst(v,#<map(lambda(tuple(x,b),x),getCache(cache,`step_cache_num))>,step);
- match step {
- case BSP(`n,`s,`k0,_,...as):
- // the initial values of all data sources
- Trees sources = #[];
- for ( Tree x: as )
- if (!x.equals(v))
- sources = sources.append(x);
- Tree res = mkBSPL(#<lambda(tuple(cache,ms,tuple(k,steps,firstp,S),peer),
- let(ignore,if(firstp,setCache(cache,`step_cache_num,S,0),0),
- let(tuple(ts,kk,step_end),
- apply(`s,tuple(cache,ms,k,peer)),
- if(step_end, // end of repeat step
- setCache(cache,o_,map(lambda(tuple(x,b),x),getCache(cache,`step_cache_num)),
- setCache(cache,`step_cache_num,
- distribute(peer,getCache(cache,`step_cache_num)),
- tuple(bag(),
- tuple(`k0,callM(plus,`plusM,steps,1),false,bag()),
- if(callM(gt,`gtM,steps,`max),
- TRUE(), // if # of steps > limit, exit
- callM(not,`notM, // ... else check the stopping condition
- aggregate(lambda(tuple(x,y),callM(or,`orM,x,y)),
- false,
- map(lambda(tuple(x,bb),bb),
- getCache(cache,`step_cache_num)))))))),
- tuple(ts,tuple(kk,steps,false,bag()),FALSE())))))>,
- #<tuple(`k0,1,true,map(lambda(x,tuple(x,false)),`ds))>,
- #<false>,
- sources);
- return #<Collect(`res)>; // must return a memory bag
- case `x: throw new Error("Cannot compile the repeat function: "+x);
- }
- case Closure(lambda(`v,`b),`ds,`max):
- Tree step = bspSimplify(mr2bsp(b));
- int step_cache_num = source_num(step);
- match step {
- case BSP(`n,`s,`k0,_,...as):
- Tree ds_source = mr2bsp(ds);
- int ds_cache_num = source_num(ds_source);
- ds_source = set_cache_num(ds_source,step_cache_num);
- // the initial values of all data sources
- Trees sources = #[`ds_source];
- Tree step_input = #<0>;
- for ( Tree x: as )
- match x {
- case BSPSource(`j,`y):
- if (y.equals(v))
- step_input = j;
- else sources = sources.append(x);
- case _: sources = sources.append(x);
- };
- s = subst_getCache_num((int)((LongLeaf)step_input).value(),step_cache_num,s);
- return mkBSPL(step_cache_num,
- #<lambda(tuple(cache,ms,tuple(k,steps,len),peer),
- let(tuple(ts,kk,step_end),
- apply(`s,tuple(cache,ms,k,peer)),
- if(step_end, // end of repeat step
- let(newLen,callM(count,`countM,getCache(cache,`step_cache_num)),
- tuple(bag(),
- tuple(`k0,callM(plus,`plusM,steps,1),newLen),
- if(callM(gt,`gtM,steps,`max),
- TRUE(), // if # of steps > limit, exit
- // ... else check if the new size is the same as the old size
- callM(geq,`geqM,len,newLen)))),
- tuple(ts,tuple(kk,steps,len),FALSE()))))>,
- #<tuple(`k0,1,callM(coerce,`coerceM,0,4))>,
- #<false>,
- sources);
- case `x: throw new Error("Cannot compile the closure function: "+x);
- }
- case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`max):
- Tree[] steps = new Tree[vs.length()];
- Tree[] inits = new Tree[vs.length()];
- int[] cache_num = new int[vs.length()];
- Tree[] k = new Tree[vs.length()];
- Trees sources = #[];
- Trees all_cache = #[];
- for ( int i = 0; i < vs.length(); i++ ) {
- inits[i] = mr2bsp(ss.nth(i));
- sources = sources.append(inits[i]);
- cache_num[i] = source_num(inits[i]);
- all_cache = all_cache.append(#<getCache(cache,`(cache_num[i]))>);
- };
- for ( int i = 0; i < vs.length(); i++ )
- match bspSimplify(mr2bsp(bs.nth(i))) {
- case BSP(`n,`s,`k0,_,...as):
- steps[i] = subst_setCache_num((int)((LongLeaf)n).value(),cache_num[i],s);
- k[i] = k0;
- loop: for ( Tree x: as )
- match x {
- case BSPSource(`m,`w):
- if (!vs.member(w))
- fail;
- for ( int j = 0; j < vs.length(); j++ )
- if (w.equals(vs.nth(j)))
- steps[i] = subst_getCache_num((int)((LongLeaf)m).value(),cache_num[j],steps[i]);
- case BSPSource(`n1,`d1):
- for ( Tree y: sources )
- match y {
- case BSPSource(`n2,`d2):
- if (d1.equals(d2)) {
- steps[i] = subst_getCache_num((int)((LongLeaf)n1).value(),
- (int)((LongLeaf)n2).value(),steps[i]);
- continue loop;
- }
- };
- sources = sources.append(x);
- case _: sources = sources.append(x);
- };
- };
- Tree code = #<tuple(BAG(),tuple(`(k[0]),0,1),TRUE())>;
- for ( int i = 0; i < vs.length(); i++ )
- code = #<if(callM(eq,`eqM,i,`i),
- let(tuple(ts,kk,step_end),
- apply(`(steps[i]),tuple(cache,ms,k,peer)),
- if(step_end, // end of repeat step
- `((i+1 < vs.length())
- ? #<tuple(bag(),tuple(`(k[i+1]),`(i+1),steps),FALSE())>
- : #<tuple(bag(),
- tuple(`(k[0]),0,callM(plus,`plusM,steps,1)),
- if(callM(gt,`gtM,steps,`max), // if # of steps > limit, exit
- TRUE(),
- FALSE()))>),
- tuple(ts,tuple(kk,i,steps),FALSE()))),
- `code)>;
- return mkBSPL(cache_num,
- #<lambda(tuple(cache,ms,tuple(k,i,steps),peer),`code)>,
- #<tuple(`(k[0]),0,2)>,
- #<false>,
- sources);
- case `f(...as):
- Trees bs = #[];
- for ( Tree a: as )
- bs = bs.append(mr2bsp(a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- /** simplify the BSP plan by fusing consequtive BSP plans */
- private static Tree bspSimplify ( Tree e ) {
- match e {
- case BSP(`n,`s2,`k2,`o,...r,BSP(_,`s1,`k1,_,...s),...t):
- Trees ys = r.append(t);
- Trees ns = #[];
- loop: for ( Tree x: s )
- match x {
- case BSPSource(`n1,`d1):
- for ( Tree y: ys )
- match y {
- case BSPSource(`n2,`d2):
- if (d1.equals(d2)) {
- s1 = subst_getCache_num((int)((LongLeaf)n1).value(),
- (int)((LongLeaf)n2).value(),s1);
- continue loop;
- }
- };
- ns = ns.append(x);
- case _: ns = ns.append(x);
- };
- return bspSimplify(mkBSPL((int)((LongLeaf)n).value(),
- #<lambda(tuple(cache,ms,tuple(first,k),peer),
- if(first,
- let(tuple(ts,kk,b),apply(`s1,tuple(cache,ms,k,peer)),
- let(exit,
- synchronize(peer,b), // poll all peers: do you want to exit?
- // all peers must aggree to exit the inner BSP
- // and proceed to the outer BSP
- tuple(ts,tuple(callM(not,`notM,exit),
- if(exit,`k2,kk)),FALSE()))),
- let(tuple(ts,kk,bb),apply(`s2,tuple(cache,ms,k,peer)),
- tuple(ts,tuple(false,kk),bb))))>,
- #<tuple(true,`k1)>,
- o,
- #[...r,...ns,...t]));
- case `f(...as):
- Trees bs = #[];
- for ( Tree a: as )
- bs = bs.append(bspSimplify(a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- private static Tree post_simplify_plan ( Tree e ) {
- match e {
- case setCache(`cache,`a,`v,`ret):
- return post_simplify_plan(#<setNth(`cache,`a,materialize(`v),`ret)>);
- case getCache(`cache,`a,...as):
- Tree z = #<nth(`cache,`a)>;
- for ( Tree x: as )
- z = #<callM(union,`unionM,`z,nth(`cache,`x))>;
- return z;
- case `f(...as):
- Trees bs = #[];
- for (Tree a: as)
- bs = bs.append(post_simplify_plan(a));
- return #<`f(...bs)>;
- };
- return e;
- }
-
- /** construct and simplify the BSP plan from a physical plan
- * @param plan the physical plan
- * @return the BSP plan
- */
- public static Tree constructBSPplan ( Tree plan ) {
- return post_simplify_plan(bspSimplify(mr2bsp(preprocess(plan))));
- }
-}
[26/26] git commit: MRQL-32: Refactoring directory structure for
Eclipse
Posted by fe...@apache.org.
MRQL-32: Refactoring directory structure for Eclipse
Project: http://git-wip-us.apache.org/repos/asf/incubator-mrql/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mrql/commit/1adaa71c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mrql/tree/1adaa71c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mrql/diff/1adaa71c
Branch: refs/heads/master
Commit: 1adaa71cb490d5a7782c8ef133115bab6e70e1c1
Parents: 44c44bd
Author: fegaras <fe...@cse.uta.edu>
Authored: Thu Mar 13 09:23:30 2014 -0500
Committer: fegaras <fe...@cse.uta.edu>
Committed: Thu Mar 13 09:23:30 2014 -0500
----------------------------------------------------------------------
BSP/pom.xml | 159 --
Gen/pom.xml | 88 --
MapReduce/pom.xml | 193 ---
Spark/pom.xml | 152 --
bsp/pom.xml | 158 ++
.../org/apache/mrql/BSPBinaryInputFormat.java | 60 +
.../main/java/org/apache/mrql/BSPEvaluator.gen | 240 ++++
.../apache/mrql/BSPGeneratorInputFormat.java | 89 ++
.../org/apache/mrql/BSPMRQLFileInputFormat.java | 107 ++
.../org/apache/mrql/BSPParsedInputFormat.java | 102 ++
bsp/src/main/java/org/apache/mrql/BSPPlan.java | 507 +++++++
.../java/org/apache/mrql/MultipleBSPInput.java | 55 +
core/pom.xml | 26 +-
.../org/apache/mrql/AlgebraicOptimization.gen | 315 ++++
.../main/java/org/apache/mrql/BSPTranslator.gen | 753 ++++++++++
core/src/main/java/org/apache/mrql/Bag.java | 578 ++++++++
.../main/java/org/apache/mrql/BagIterator.java | 27 +
.../java/org/apache/mrql/BinaryDataSource.java | 36 +
.../java/org/apache/mrql/ClassImporter.java | 202 +++
core/src/main/java/org/apache/mrql/Compiler.gen | 584 ++++++++
core/src/main/java/org/apache/mrql/Config.java | 277 ++++
core/src/main/java/org/apache/mrql/DataSet.java | 109 ++
.../main/java/org/apache/mrql/DataSource.java | 195 +++
.../main/java/org/apache/mrql/Environment.java | 44 +
.../main/java/org/apache/mrql/Evaluator.java | 152 ++
.../src/main/java/org/apache/mrql/Function.java | 34 +
.../org/apache/mrql/GeneratorDataSource.java | 49 +
.../main/java/org/apache/mrql/Interpreter.gen | 833 +++++++++++
core/src/main/java/org/apache/mrql/Inv.java | 65 +
core/src/main/java/org/apache/mrql/JSON.cup | 60 +
core/src/main/java/org/apache/mrql/JSON.lex | 68 +
.../java/org/apache/mrql/JsonFormatParser.java | 91 ++
.../main/java/org/apache/mrql/JsonSplitter.java | 154 ++
core/src/main/java/org/apache/mrql/Lambda.java | 52 +
.../main/java/org/apache/mrql/LineParser.gen | 232 +++
.../main/java/org/apache/mrql/MRContainer.java | 153 ++
core/src/main/java/org/apache/mrql/MRData.java | 27 +
core/src/main/java/org/apache/mrql/MRQL.java | 58 +
.../org/apache/mrql/MRQLFileInputFormat.java | 36 +
core/src/main/java/org/apache/mrql/MR_bool.java | 72 +
core/src/main/java/org/apache/mrql/MR_byte.java | 73 +
core/src/main/java/org/apache/mrql/MR_char.java | 71 +
.../main/java/org/apache/mrql/MR_dataset.java | 52 +
.../main/java/org/apache/mrql/MR_double.java | 75 +
.../src/main/java/org/apache/mrql/MR_float.java | 77 +
core/src/main/java/org/apache/mrql/MR_int.java | 74 +
core/src/main/java/org/apache/mrql/MR_long.java | 77 +
.../java/org/apache/mrql/MR_more_bsp_steps.java | 44 +
.../src/main/java/org/apache/mrql/MR_short.java | 71 +
.../main/java/org/apache/mrql/MR_string.java | 77 +
core/src/main/java/org/apache/mrql/MR_sync.java | 44 +
.../main/java/org/apache/mrql/MR_variable.java | 46 +
core/src/main/java/org/apache/mrql/Main.java | 173 +++
.../java/org/apache/mrql/MapReduceAlgebra.java | 810 +++++++++++
.../java/org/apache/mrql/Materialization.gen | 134 ++
.../main/java/org/apache/mrql/MethodInfo.java | 57 +
.../main/java/org/apache/mrql/Normalization.gen | 406 ++++++
.../java/org/apache/mrql/ParsedDataSource.java | 62 +
core/src/main/java/org/apache/mrql/Parser.java | 32 +
core/src/main/java/org/apache/mrql/Plan.java | 362 +++++
.../java/org/apache/mrql/PlanGeneration.gen | 716 +++++++++
core/src/main/java/org/apache/mrql/Printer.gen | 483 +++++++
.../src/main/java/org/apache/mrql/QueryPlan.gen | 999 +++++++++++++
.../java/org/apache/mrql/Simplification.gen | 389 +++++
.../java/org/apache/mrql/SystemFunctions.java | 467 ++++++
core/src/main/java/org/apache/mrql/Test.java | 127 ++
core/src/main/java/org/apache/mrql/TopLevel.gen | 293 ++++
.../main/java/org/apache/mrql/Translator.gen | 445 ++++++
core/src/main/java/org/apache/mrql/Tuple.java | 202 +++
.../main/java/org/apache/mrql/TypeInference.gen | 1356 ++++++++++++++++++
core/src/main/java/org/apache/mrql/Union.java | 84 ++
.../main/java/org/apache/mrql/XMLParser.java | 109 ++
.../main/java/org/apache/mrql/XMLSplitter.java | 158 ++
.../main/java/org/apache/mrql/XPathParser.java | 692 +++++++++
core/src/main/java/org/apache/mrql/mrql.cgen | 365 +++++
core/src/main/java/org/apache/mrql/mrql.lex | 186 +++
gen/pom.xml | 87 ++
.../java/org/apache/mrql/gen/Condition.java | 37 +
.../main/java/org/apache/mrql/gen/Crypt.java | 58 +
.../java/org/apache/mrql/gen/DoubleLeaf.java | 42 +
.../main/java/org/apache/mrql/gen/LongLeaf.java | 52 +
gen/src/main/java/org/apache/mrql/gen/Main.java | 61 +
gen/src/main/java/org/apache/mrql/gen/Meta.java | 260 ++++
gen/src/main/java/org/apache/mrql/gen/Node.java | 77 +
.../java/org/apache/mrql/gen/StringLeaf.java | 47 +
.../java/org/apache/mrql/gen/SymbolTable.java | 193 +++
gen/src/main/java/org/apache/mrql/gen/Tree.java | 93 ++
.../main/java/org/apache/mrql/gen/Trees.java | 194 +++
.../java/org/apache/mrql/gen/VariableLeaf.java | 52 +
gen/src/main/java/org/apache/mrql/gen/gen.cup | 179 +++
gen/src/main/java/org/apache/mrql/gen/gen.lex | 210 +++
mapreduce/pom.xml | 191 +++
.../org/apache/mrql/CrossProductOperation.java | 235 +++
.../java/org/apache/mrql/GroupByJoinPlan.java | 464 ++++++
.../java/org/apache/mrql/JoinOperation.java | 405 ++++++
.../java/org/apache/mrql/MapJoinOperation.java | 213 +++
.../main/java/org/apache/mrql/MapOperation.java | 133 ++
.../apache/mrql/MapReduceBinaryInputFormat.java | 80 ++
.../java/org/apache/mrql/MapReduceEvaluator.gen | 296 ++++
.../mrql/MapReduceGeneratorInputFormat.java | 114 ++
.../mrql/MapReduceMRQLFileInputFormat.java | 54 +
.../org/apache/mrql/MapReduceOperation.java | 273 ++++
.../apache/mrql/MapReduceParsedInputFormat.java | 139 ++
.../java/org/apache/mrql/MapReducePlan.java | 125 ++
pom.xml | 8 +-
spark/pom.xml | 151 ++
spark/src/main/java/org/apache/mrql/MR_rdd.java | 53 +
.../java/org/apache/mrql/RDDDataSource.java | 62 +
.../org/apache/mrql/SparkBinaryInputFormat.java | 53 +
.../java/org/apache/mrql/SparkEvaluator.gen | 762 ++++++++++
.../apache/mrql/SparkGeneratorInputFormat.java | 113 ++
.../apache/mrql/SparkMRQLFileInputFormat.java | 83 ++
.../org/apache/mrql/SparkParsedInputFormat.java | 97 ++
src/main/java/BSP/BSPBinaryInputFormat.java | 60 -
src/main/java/BSP/BSPEvaluator.gen | 240 ----
src/main/java/BSP/BSPGeneratorInputFormat.java | 89 --
src/main/java/BSP/BSPMRQLFileInputFormat.java | 107 --
src/main/java/BSP/BSPParsedInputFormat.java | 102 --
src/main/java/BSP/BSPPlan.java | 507 -------
src/main/java/BSP/MultipleBSPInput.java | 55 -
.../java/MapReduce/CrossProductOperation.java | 235 ---
src/main/java/MapReduce/GroupByJoinPlan.java | 464 ------
src/main/java/MapReduce/JoinOperation.java | 405 ------
src/main/java/MapReduce/MapJoinOperation.java | 213 ---
src/main/java/MapReduce/MapOperation.java | 133 --
.../MapReduce/MapReduceBinaryInputFormat.java | 80 --
src/main/java/MapReduce/MapReduceEvaluator.gen | 296 ----
.../MapReduceGeneratorInputFormat.java | 114 --
.../MapReduce/MapReduceMRQLFileInputFormat.java | 54 -
src/main/java/MapReduce/MapReduceOperation.java | 273 ----
.../MapReduce/MapReduceParsedInputFormat.java | 139 --
src/main/java/MapReduce/MapReducePlan.java | 125 --
.../MultipleInputs/DelegatingInputFormat.java | 127 --
.../java/MultipleInputs/DelegatingMapper.java | 54 -
.../MultipleInputs/DelegatingRecordReader.java | 88 --
.../java/MultipleInputs/MultipleInputs.java | 141 --
.../java/MultipleInputs/TaggedInputSplit.java | 159 --
src/main/java/core/AlgebraicOptimization.gen | 315 ----
src/main/java/core/BSPTranslator.gen | 753 ----------
src/main/java/core/Bag.java | 578 --------
src/main/java/core/BagIterator.java | 27 -
src/main/java/core/BinaryDataSource.java | 36 -
src/main/java/core/ClassImporter.java | 202 ---
src/main/java/core/Compiler.gen | 584 --------
src/main/java/core/Config.java | 277 ----
src/main/java/core/DataSet.java | 109 --
src/main/java/core/DataSource.java | 195 ---
src/main/java/core/Environment.java | 44 -
src/main/java/core/Evaluator.java | 152 --
src/main/java/core/Function.java | 34 -
src/main/java/core/GeneratorDataSource.java | 49 -
src/main/java/core/Interpreter.gen | 833 -----------
src/main/java/core/Inv.java | 65 -
src/main/java/core/JSON.cup | 60 -
src/main/java/core/JSON.lex | 68 -
src/main/java/core/JsonFormatParser.java | 91 --
src/main/java/core/JsonSplitter.java | 154 --
src/main/java/core/Lambda.java | 52 -
src/main/java/core/LineParser.gen | 232 ---
src/main/java/core/MRContainer.java | 153 --
src/main/java/core/MRData.java | 27 -
src/main/java/core/MRQL.java | 58 -
src/main/java/core/MRQLFileInputFormat.java | 36 -
src/main/java/core/MR_bool.java | 72 -
src/main/java/core/MR_byte.java | 73 -
src/main/java/core/MR_char.java | 71 -
src/main/java/core/MR_dataset.java | 52 -
src/main/java/core/MR_double.java | 75 -
src/main/java/core/MR_float.java | 77 -
src/main/java/core/MR_int.java | 74 -
src/main/java/core/MR_long.java | 77 -
src/main/java/core/MR_more_bsp_steps.java | 44 -
src/main/java/core/MR_short.java | 71 -
src/main/java/core/MR_string.java | 77 -
src/main/java/core/MR_sync.java | 44 -
src/main/java/core/MR_variable.java | 46 -
src/main/java/core/Main.java | 173 ---
src/main/java/core/MapReduceAlgebra.java | 810 -----------
src/main/java/core/Materialization.gen | 134 --
src/main/java/core/MethodInfo.java | 57 -
src/main/java/core/Normalization.gen | 406 ------
src/main/java/core/ParsedDataSource.java | 62 -
src/main/java/core/Parser.java | 32 -
src/main/java/core/Plan.java | 362 -----
src/main/java/core/PlanGeneration.gen | 716 ---------
src/main/java/core/Printer.gen | 483 -------
src/main/java/core/QueryPlan.gen | 999 -------------
src/main/java/core/Simplification.gen | 389 -----
src/main/java/core/SystemFunctions.java | 467 ------
src/main/java/core/Test.java | 127 --
src/main/java/core/TopLevel.gen | 293 ----
src/main/java/core/Translator.gen | 445 ------
src/main/java/core/Tuple.java | 202 ---
src/main/java/core/TypeInference.gen | 1356 ------------------
src/main/java/core/Union.java | 84 --
src/main/java/core/XMLParser.java | 109 --
src/main/java/core/XMLSplitter.java | 158 --
src/main/java/core/XPathParser.java | 692 ---------
src/main/java/core/mrql.cgen | 365 -----
src/main/java/core/mrql.lex | 186 ---
src/main/java/gen/Condition.java | 37 -
src/main/java/gen/Crypt.java | 58 -
src/main/java/gen/DoubleLeaf.java | 42 -
src/main/java/gen/LongLeaf.java | 52 -
src/main/java/gen/Main.java | 61 -
src/main/java/gen/Meta.java | 260 ----
src/main/java/gen/Node.java | 77 -
src/main/java/gen/StringLeaf.java | 47 -
src/main/java/gen/SymbolTable.java | 193 ---
src/main/java/gen/Tree.java | 93 --
src/main/java/gen/Trees.java | 194 ---
src/main/java/gen/VariableLeaf.java | 52 -
src/main/java/gen/gen.cup | 179 ---
src/main/java/gen/gen.lex | 210 ---
.../multipleinputs/DelegatingInputFormat.java | 127 ++
.../java/multipleinputs/DelegatingMapper.java | 54 +
.../multipleinputs/DelegatingRecordReader.java | 88 ++
.../java/multipleinputs/MultipleInputs.java | 141 ++
.../java/multipleinputs/TaggedInputSplit.java | 159 ++
src/main/java/spark/MR_rdd.java | 53 -
src/main/java/spark/RDDDataSource.java | 62 -
src/main/java/spark/SparkBinaryInputFormat.java | 53 -
src/main/java/spark/SparkEvaluator.gen | 762 ----------
.../java/spark/SparkGeneratorInputFormat.java | 113 --
.../java/spark/SparkMRQLFileInputFormat.java | 83 --
src/main/java/spark/SparkParsedInputFormat.java | 97 --
226 files changed, 22778 insertions(+), 22799 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/BSP/pom.xml
----------------------------------------------------------------------
diff --git a/BSP/pom.xml b/BSP/pom.xml
deleted file mode 100644
index 699c18c..0000000
--- a/BSP/pom.xml
+++ /dev/null
@@ -1,159 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-bsp</artifactId>
- <packaging>jar</packaging>
- <name>Apache MRQL BSP mode</name>
- <description>Apache MRQL evaluation in BSP mode on Apache Hama</description>
- <url>http://mrql.incubator.apache.org/</url>
- <inceptionYear>2013</inceptionYear>
-
- <parent>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-parent</artifactId>
- <version>0.9.1-incubating-SNAPSHOT</version>
- </parent>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-gen</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-core</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hama</groupId>
- <artifactId>hama-core</artifactId>
- <version>${hama.version}</version>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>build-helper-maven-plugin</artifactId>
- <version>1.8</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals><goal>add-source</goal></goals>
- <configuration>
- <sources>
- <source>../src/main/java/BSP</source>
- <source>${project.build.directory}/generated-sources/org/apache/mrql</source>
- </sources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <version>2.1</version>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <artifactSet>
- <includes>
- <include>org.apache.mrql:*</include>
- </includes>
- <excludes>
- <exclude>org.apache.mrql:mrql-gen</exclude>
- <exclude>org.apache.mrql:mrql-core</exclude>
- </excludes>
- </artifactSet>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
- <configuration>
- <finalName>mrql-bsp-${project.version}</finalName>
- <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
- </configuration>
- </plugin>
- <plugin>
- <artifactId>maven-antrun-plugin</artifactId>
- <version>1.7</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target>
- <mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
- <property name="compile_classpath" refid="maven.compile.classpath" />
- <fileset id="mr.gen.path" dir="../src/main/java/BSP" includes="*.gen" />
- <pathconvert pathsep=" " property="mr.gen.files" refid="mr.gen.path" />
- <java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
- <arg line="${mr.gen.files}" />
- <arg line="-o" />
- <arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
- </java>
- </target>
- </configuration>
- </execution>
- <execution>
- <id>validate</id>
- <phase>test</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target name="validate_hama" if="tests" description="Validate all test queries on Apache Hama">
- <property name="runtime_classpath" refid="maven.runtime.classpath" />
- <echo message="Evaluating test queries in memory (BSP mode):" />
- <java classname="org.apache.mrql.Test" classpath="${runtime_classpath}" dir=".." fork="yes">
- <arg line="-bsp" />
- <arg file="../tests/queries" />
- <arg file="../tests/results/bsp-memory" />
- <arg file="../tests/error_log.txt" />
- </java>
- <echo message="Evaluating test queries in Apache Hama local mode:" />
- <java classname="org.apache.mrql.Test" classpath="../lib/mrql-bsp-${project.version}.jar:${runtime_classpath}" dir=".." fork="yes" error="/dev/null">
- <arg line="-local" />
- <arg line="-bsp" />
- <arg file="../tests/queries" />
- <arg file="../tests/results/bsp" />
- <arg file="../tests/error_log.txt" />
- </java>
- </target>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/Gen/pom.xml
----------------------------------------------------------------------
diff --git a/Gen/pom.xml b/Gen/pom.xml
deleted file mode 100644
index bb90718..0000000
--- a/Gen/pom.xml
+++ /dev/null
@@ -1,88 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-gen</artifactId>
- <packaging>jar</packaging>
- <name>Apache MRQL gen</name>
- <description>Apache MRQL Gen: A Java Package for Constructing and Manipulating Abstract Syntax Trees</description>
- <url>http://mrql.incubator.apache.org/</url>
- <inceptionYear>2013</inceptionYear>
-
- <parent>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-parent</artifactId>
- <version>0.9.1-incubating-SNAPSHOT</version>
- </parent>
-
- <build>
- <sourceDirectory>../src/main/java/gen</sourceDirectory>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
- <configuration>
- <finalName>mrql-gen-${project.version}</finalName>
- <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
- <useDefaultManifestFile>true</useDefaultManifestFile>
- </configuration>
- </plugin>
- <plugin>
- <groupId>de.jflex</groupId>
- <artifactId>maven-jflex-plugin</artifactId>
- <version>1.4.3</version>
- <executions>
- <execution>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <lexDefinitions>
- <lexDefinition>../src/main/java/gen/gen.lex</lexDefinition>
- </lexDefinitions>
- <outputDirectory>${project.build.directory}/generated-sources/parsers</outputDirectory>
- <verbose>false</verbose>
- <backup>false</backup>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <artifactId>maven-antrun-plugin</artifactId>
- <version>1.7</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target>
- <taskdef name="cup" classname="java_cup.anttask.CUPTask" classpathref="maven.compile.classpath" />
- <cup srcfile="src/main/java/gen/gen.cup" destdir="${project.build.directory}/generated-sources/parsers" nosummary="true" parser="GenParser" symbols="GenSym" />
- </target>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/MapReduce/pom.xml
----------------------------------------------------------------------
diff --git a/MapReduce/pom.xml b/MapReduce/pom.xml
deleted file mode 100644
index ab36e49..0000000
--- a/MapReduce/pom.xml
+++ /dev/null
@@ -1,193 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-mr</artifactId>
- <packaging>jar</packaging>
- <name>Apache MRQL MapReduce mode</name>
- <description>Apache MRQL evaluation in MapReduce mode on Apache Hadoop</description>
- <url>http://mrql.incubator.apache.org/</url>
- <inceptionYear>2013</inceptionYear>
-
- <parent>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-parent</artifactId>
- <version>0.9.1-incubating-SNAPSHOT</version>
- </parent>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-gen</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-core</artifactId>
- <version>${project.version}</version>
- </dependency>
- <!-- needed for testing in local mode -->
- <dependency>
- <groupId>commons-io</groupId>
- <artifactId>commons-io</artifactId>
- <version>2.4</version>
- </dependency>
- </dependencies>
-
- <profiles>
- <profile>
- <id>MultipleInputs</id>
- <activation>
- <property>
- <name>MultipleInputs</name>
- </property>
- </activation>
- <build>
- <plugins>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>build-helper-maven-plugin</artifactId>
- <version>1.8</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals><goal>add-source</goal></goals>
- <configuration>
- <sources>
- <source>../src/main/java/MultipleInputs</source>
- <source>../src/main/java/MapReduce</source>
- <source>${project.build.directory}/generated-sources/org/apache/mrql</source>
- </sources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
- </profile>
- </profiles>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>build-helper-maven-plugin</artifactId>
- <version>1.8</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals><goal>add-source</goal></goals>
- <configuration>
- <sources>
- <source>../src/main/java/MapReduce</source>
- <source>${project.build.directory}/generated-sources/org/apache/mrql</source>
- </sources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <version>2.1</version>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <artifactSet>
- <includes>
- <include>org.apache.mrql:*</include>
- </includes>
- <excludes>
- <exclude>org.apache.mrql:mrql-gen</exclude>
- <exclude>org.apache.mrql:mrql-core</exclude>
- </excludes>
- </artifactSet>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
- <configuration>
- <finalName>mrql-mr-${project.version}</finalName>
- <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
- <useDefaultManifestFile>true</useDefaultManifestFile>
- </configuration>
- </plugin>
- <plugin>
- <artifactId>maven-antrun-plugin</artifactId>
- <version>1.7</version>
- <executions>
- <execution>
- <id>gen</id>
- <phase>generate-sources</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target>
- <mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
- <property name="compile_classpath" refid="maven.compile.classpath" />
- <fileset id="mr.gen.path" dir="../src/main/java/MapReduce" includes="*.gen" />
- <pathconvert pathsep=" " property="mr.gen.files" refid="mr.gen.path" />
- <java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
- <arg line="${mr.gen.files}" />
- <arg line="-o" />
- <arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
- </java>
- </target>
- </configuration>
- </execution>
- <execution>
- <id>validate</id>
- <phase>test</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target name="validate_hadoop" if="tests" description="Validate all test queries on Apache Hadoop">
- <property name="runtime_classpath" refid="maven.runtime.classpath" />
- <echo message="Evaluating test queries in memory (Map-Reduce mode):" />
- <java classname="org.apache.mrql.Test" classpath="${runtime_classpath}" dir=".." fork="yes">
- <arg file="../tests/queries" />
- <arg file="../tests/results/mr-memory" />
- <arg file="../tests/error_log.txt" />
- </java>
- <echo message="Evaluating test queries in Apache Hadoop local mode:" />
- <java classname="org.apache.mrql.Test" classpath="../lib/mrql-mr-${project.version}.jar:${runtime_classpath}" dir=".." fork="yes" error="/dev/null">
- <arg line="-local" />
- <arg file="../tests/queries" />
- <arg file="../tests/results/hadoop" />
- <arg file="../tests/error_log.txt" />
- </java>
- </target>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/Spark/pom.xml
----------------------------------------------------------------------
diff --git a/Spark/pom.xml b/Spark/pom.xml
deleted file mode 100644
index a8dcb27..0000000
--- a/Spark/pom.xml
+++ /dev/null
@@ -1,152 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-spark</artifactId>
- <packaging>jar</packaging>
- <name>Apache MRQL Spark mode</name>
- <description>Apache MRQL evaluation in Spark mode on Apache Spark</description>
- <url>http://mrql.incubator.apache.org/</url>
- <inceptionYear>2013</inceptionYear>
-
- <parent>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-parent</artifactId>
- <version>0.9.1-incubating-SNAPSHOT</version>
- </parent>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-gen</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.mrql</groupId>
- <artifactId>mrql-core</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-core_${scala.version}</artifactId>
- <version>${spark.version}</version>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>build-helper-maven-plugin</artifactId>
- <version>1.8</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals><goal>add-source</goal></goals>
- <configuration>
- <sources>
- <source>../src/main/java/spark</source>
- <source>${project.build.directory}/generated-sources/org/apache/mrql</source>
- </sources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <version>2.1</version>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <artifactSet>
- <includes>
- <include>org.apache.mrql:*</include>
- </includes>
- <excludes>
- <exclude>org.apache.mrql:mrql-gen</exclude>
- <exclude>org.apache.mrql:mrql-core</exclude>
- </excludes>
- </artifactSet>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <version>2.4</version>
- <configuration>
- <finalName>mrql-spark-${project.version}</finalName>
- <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
- </configuration>
- </plugin>
- <plugin>
- <artifactId>maven-antrun-plugin</artifactId>
- <version>1.7</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target>
- <mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
- <property name="compile_classpath" refid="maven.compile.classpath" />
- <fileset id="mr.gen.path" dir="../src/main/java/spark" includes="*.gen" />
- <pathconvert pathsep=" " property="mr.gen.files" refid="mr.gen.path" />
- <java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
- <arg line="${mr.gen.files}" />
- <arg line="-o" />
- <arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
- </java>
- </target>
- </configuration>
- </execution>
- <execution>
- <id>validate</id>
- <phase>test</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target name="validate_spark" if="tests" description="Validate all test queries on Apache Spark">
- <property name="runtime_classpath" refid="maven.runtime.classpath" />
- <echo message="Evaluating test queries in Apache Spark local mode:" />
- <java classname="org.apache.mrql.Test" classpath="../lib/mrql-spark-${project.version}.jar:${runtime_classpath}" dir=".." fork="yes" error="/dev/null">
- <arg line="-local" />
- <arg line="-spark" />
- <arg file="../tests/queries" />
- <arg file="../tests/results/spark" />
- <arg file="../tests/error_log.txt" />
- </java>
- </target>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/pom.xml
----------------------------------------------------------------------
diff --git a/bsp/pom.xml b/bsp/pom.xml
new file mode 100644
index 0000000..0c07d7f
--- /dev/null
+++ b/bsp/pom.xml
@@ -0,0 +1,158 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-bsp</artifactId>
+ <packaging>jar</packaging>
+ <name>Apache MRQL BSP mode</name>
+ <description>Apache MRQL evaluation in BSP mode on Apache Hama</description>
+ <url>http://mrql.incubator.apache.org/</url>
+ <inceptionYear>2013</inceptionYear>
+
+ <parent>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-parent</artifactId>
+ <version>0.9.1-incubating-SNAPSHOT</version>
+ </parent>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-gen</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hama</groupId>
+ <artifactId>hama-core</artifactId>
+ <version>${hama.version}</version>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.8</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals><goal>add-source</goal></goals>
+ <configuration>
+ <sources>
+ <source>${project.build.directory}/generated-sources</source>
+ </sources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>2.1</version>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <artifactSet>
+ <includes>
+ <include>org.apache.mrql:*</include>
+ </includes>
+ <excludes>
+ <exclude>org.apache.mrql:mrql-gen</exclude>
+ <exclude>org.apache.mrql:mrql-core</exclude>
+ </excludes>
+ </artifactSet>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.4</version>
+ <configuration>
+ <finalName>mrql-bsp-${project.version}</finalName>
+ <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <version>1.7</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target>
+ <mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
+ <property name="compile_classpath" refid="maven.compile.classpath" />
+ <fileset id="mr.gen.path" dir="src/main/java/org/apache/mrql" includes="*.gen" />
+ <pathconvert pathsep=" " property="mr.gen.files" refid="mr.gen.path" />
+ <java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
+ <arg line="${mr.gen.files}" />
+ <arg line="-o" />
+ <arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
+ </java>
+ </target>
+ </configuration>
+ </execution>
+ <execution>
+ <id>validate</id>
+ <phase>test</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target name="validate_hama" if="tests" description="Validate all test queries on Apache Hama">
+ <property name="runtime_classpath" refid="maven.runtime.classpath" />
+ <echo message="Evaluating test queries in memory (BSP mode):" />
+ <java classname="org.apache.mrql.Test" classpath="${runtime_classpath}" dir=".." fork="yes">
+ <arg line="-bsp" />
+ <arg file="../tests/queries" />
+ <arg file="../tests/results/bsp-memory" />
+ <arg file="../tests/error_log.txt" />
+ </java>
+ <echo message="Evaluating test queries in Apache Hama local mode:" />
+ <java classname="org.apache.mrql.Test" classpath="../lib/mrql-bsp-${project.version}.jar:${runtime_classpath}" dir=".." fork="yes" error="/dev/null">
+ <arg line="-local" />
+ <arg line="-bsp" />
+ <arg file="../tests/queries" />
+ <arg file="../tests/results/bsp" />
+ <arg file="../tests/error_log.txt" />
+ </java>
+ </target>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/BSPBinaryInputFormat.java
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/BSPBinaryInputFormat.java b/bsp/src/main/java/org/apache/mrql/BSPBinaryInputFormat.java
new file mode 100644
index 0000000..8c29a6d
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/BSPBinaryInputFormat.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.Trees;
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hama.bsp.*;
+import org.apache.hama.HamaConfiguration;
+
+
+/** Input format for hadoop sequence files */
+final public class BSPBinaryInputFormat extends BSPMRQLFileInputFormat {
+ public static class BinaryInputRecordReader extends SequenceFileRecordReader<MRContainer,MRContainer> {
+ final MRContainer result = new MRContainer();
+ final MRData source_num_data;
+ final int source_number;
+
+ public BinaryInputRecordReader ( FileSplit split,
+ BSPJob job,
+ int source_number ) throws IOException {
+ super(BSPPlan.getConfiguration(job),split);
+ this.source_number = source_number;
+ source_num_data = new MR_int(source_number);
+ }
+
+ @Override
+ public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
+ boolean b = super.next(key,result);
+ value.set(new Tuple(source_num_data,result.data()));
+ return b;
+ }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split,
+ BSPJob job ) throws IOException {
+ Configuration conf = BSPPlan.getConfiguration(job);
+ String path = ((FileSplit)split).getPath().toString();
+ BinaryDataSource ds = (BinaryDataSource)DataSource.get(path,conf);
+ return new BinaryInputRecordReader((FileSplit)split,job,ds.source_num);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/BSPEvaluator.gen
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/BSPEvaluator.gen b/bsp/src/main/java/org/apache/mrql/BSPEvaluator.gen
new file mode 100644
index 0000000..e8fb859
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/BSPEvaluator.gen
@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.List;
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hama.HamaConfiguration;
+
+
+/** Evaluate a physical plan in BSP mode using Apache Hama */
+final public class BSPEvaluator extends Evaluator {
+
+ /** initialize the BSP evaluator */
+ final public void init ( Configuration conf ) {
+ Config.bsp_mode = true;
+ if (Config.hadoop_mode)
+ if (Config.local_mode) {
+ conf.set("bsp.master.address","local");
+ conf.set("hama.zookeeper.quorum","localhost");
+ conf.setInt("bsp.local.tasks.maximum",Config.nodes);
+ conf.set("fs.default.name","file:///");
+ } else {
+ conf.set("bsp.master.address",System.getenv("BSP_MASTER_ADDRESS"));
+ conf.set("hama.zookeeper.quorum",System.getenv("HAMA_ZOOKEEPER_QUORUM"));
+ conf.setInt("bsp.local.tasks.maximum",Config.nodes);
+ conf.set("fs.default.name",System.getenv("FS_DEFAULT_NAME"));
+ }
+ }
+
+ final public void initialize_query () {
+ }
+
+ /** shutdown the BSP evaluator */
+ final public void shutdown ( Configuration conf ) {
+ }
+
+ final public Configuration new_configuration () {
+ return new HamaConfiguration();
+ }
+
+ final public MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
+ if (Config.hadoop_mode)
+ return BSPPlan.BSPop.synchronize(peerName,mr_exit);
+ else return mr_exit;
+ }
+
+ final public Bag distribute ( MR_string peerName, Bag s ) {
+ if (Config.hadoop_mode)
+ return BSPPlan.BSPop.distribute(peerName,s);
+ else return s;
+ }
+
+ /** Evaluate MRQL physical operators using BSP
+ * @param plan the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a DataSet (stored in HDFS)
+ */
+ final public MRData bsp ( Tree plan, Environment env ) throws Exception {
+ match plan {
+ case BSP(tuple(...ns),`superstep,`init_state,`o,...S):
+ int[] os = new int[ns.length()];
+ for ( int i = 0; i < os.length; i++ )
+ os[i] = (int)((LongLeaf)ns.nth(i)).value();
+ DataSet ds = eval(S.head(),env);
+ for ( Tree s: S.tail() )
+ ds.merge(eval(s,env));
+ return BSPPlan.BSP(os,
+ closure(superstep,env),
+ init_state,
+ o.equals(#<true>),
+ ds);
+ case BSP(`n,`superstep,`init_state,`o,...S):
+ DataSet ds = eval(S.head(),env);
+ for ( Tree s: S.tail() )
+ ds.merge(eval(s,env));
+ return BSPPlan.BSP(new int[]{(int)((LongLeaf)n).value()},
+ closure(superstep,env),
+ init_state,
+ o.equals(#<true>),
+ ds);
+ }
+ throw new Error("Cannot perform the BSP plan: "+plan);
+ }
+
+ /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
+ final public Class<? extends MRQLFileInputFormat> parsedInputFormat () {
+ return BSPParsedInputFormat.class;
+ }
+
+ /** return the FileInputFormat for binary files */
+ final public Class<? extends MRQLFileInputFormat> binaryInputFormat () {
+ return BSPBinaryInputFormat.class;
+ }
+
+ /** return the FileInputFormat for data generator files */
+ final public Class<? extends MRQLFileInputFormat> generatorInputFormat () {
+ return BSPGeneratorInputFormat.class;
+ }
+
+ /** The Aggregate physical operator
+ * @param acc_fnc the accumulator function from (T,T) to T
+ * @param zero the zero element of type T
+ * @param plan the plan that constructs the dataset that contains the bag of values {T}
+ * @param env contains bindings fro variables to values (MRData)
+ * @return the aggregation result of type T
+ */
+ final public MRData aggregate ( Tree acc_fnc,
+ Tree zero,
+ Tree plan,
+ Environment env ) throws Exception {
+ match plan {
+ case BSP(`n,`superstep,`init_state,`o,...S):
+ DataSet ds = eval(S.head(),env,"-");
+ for ( Tree s: S.tail() )
+ ds.merge(eval(s,env,"-"));
+ return BSPPlan.BSPaggregate((int)((LongLeaf)n).value(),
+ closure(superstep,env),
+ init_state,
+ closure(acc_fnc,env),
+ zero,
+ ds);
+ }
+ throw new Error("Cannot perform the aggregation: "+plan);
+ }
+
+ final public Tuple loop ( Tree e, Environment env ) {
+ throw new Error("The BSP Loop was supposed to be translated to a BSP task");
+ }
+
+ final public DataSet eval ( final Tree e,
+ final Environment env,
+ final String counter ) {
+ return eval(e,env);
+ }
+
+ /** Evaluate MRQL BSP physical operators using Hama (returns a DataSet)
+ * and print tracing info
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a DataSet (stored in HDFS)
+ */
+ final public DataSet eval ( final Tree e, final Environment env ) {
+ if (Config.trace_execution) {
+ tab_count += 3;
+ System.out.println(tabs(tab_count)+print_query(e));
+ };
+ DataSet res = evalD(e,env);
+ if (Config.trace_execution)
+ try {
+ System.out.println(tabs(tab_count)
+ +"-> "+res.take(Config.max_bag_size_print));
+ tab_count -= 3;
+ } catch (Exception ex) {
+ throw new Error("Cannot collect the operator output: "+e);
+ };
+ return res;
+ }
+
+ /** Evaluate MRQL BSP physical operators using Hama (returns a DataSet)
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a DataSet (stored in HDFS)
+ */
+ final public DataSet evalD ( final Tree e, final Environment env ) {
+ try {
+ match e {
+ case BSPSource(`n,BinarySource(`file,_)):
+ return Plan.binarySource((int)((LongLeaf)n).value(),file.stringValue());
+ case BSPSource(`n,ParsedSource(`parser,`file,...args)):
+ Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ return Plan.parsedSource((int)((LongLeaf)n).value(),
+ p,((MR_string)evalE(file,env)).get(),args);
+ case BSPSource(`n,Generator(`min,`max,`size)):
+ return Plan.generator((int)((LongLeaf)n).value(),
+ ((MR_long)evalE(min,env)).get(),
+ ((MR_long)evalE(max,env)).get(),
+ ((MR_long)evalE(size,env)).get());
+ case Merge(`x,`y):
+ return Plan.merge(eval(x,env),eval(y,env));
+ case Dump(`s):
+ return Plan.fileCache((Bag)evalE(s,env));
+ case apply(`f,`arg):
+ if (!f.is_variable())
+ return ((MR_dataset)evalF(f,env).eval(evalE(arg))).dataset();
+ MRData fnc = variable_lookup(f.toString(),global_env);
+ if (fnc == null)
+ throw new Error("Unknown function: "+f);
+ MRData t = evalE(arg,env);
+ if (!(t instanceof Tuple))
+ throw new Error("Expected a tuple in function application: "+t);
+ return ((MR_dataset)((Lambda)fnc).lambda().eval(t)).dataset();
+ case `v:
+ if (!v.is_variable())
+ fail;
+ MRData x = variable_lookup(v.toString(),env);
+ if (x != null)
+ if (x instanceof MR_dataset)
+ return ((MR_dataset)x).dataset();
+ x = variable_lookup(v.toString(),global_env);
+ if (x != null)
+ if (x instanceof MR_dataset)
+ return ((MR_dataset)x).dataset();
+ throw new Error("Variable "+v+" is not bound");
+ };
+ MRData d = bsp(e,env);
+ if (d instanceof MR_dataset)
+ return ((MR_dataset)d).dataset();
+ throw new Error("Cannot evaluate the BSP plan: "+e);
+ } catch (Error msg) {
+ if (!Config.trace)
+ throw new Error(msg.getMessage());
+ System.err.println(msg.getMessage());
+ throw new Error("Evaluation error in: "+print_query(e));
+ } catch (Exception ex) {
+ System.err.println(ex.getMessage());
+ ex.printStackTrace();
+ throw new Error("Evaluation error in: "+print_query(e));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/BSPGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/BSPGeneratorInputFormat.java b/bsp/src/main/java/org/apache/mrql/BSPGeneratorInputFormat.java
new file mode 100644
index 0000000..bdee64e
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/BSPGeneratorInputFormat.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.Trees;
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hama.bsp.*;
+
+
+/** the FileInputFormat for data generators: it creates HDFS files, where each file contains
+ * an (offset,size) pair that generates the range of values [offset,offset+size] */
+final public class BSPGeneratorInputFormat extends BSPMRQLFileInputFormat {
+ public static class GeneratorRecordReader implements RecordReader<MRContainer,MRContainer> {
+ final long offset;
+ final long size;
+ final int source_number;
+ final MRData source_num_data;
+ long index;
+ SequenceFile.Reader reader;
+
+ public GeneratorRecordReader ( FileSplit split,
+ int source_number,
+ BSPJob job ) throws IOException {
+ Configuration conf = BSPPlan.getConfiguration(job);
+ Path path = split.getPath();
+ FileSystem fs = path.getFileSystem(conf);
+ reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
+ MRContainer key = new MRContainer();
+ MRContainer value = new MRContainer();
+ reader.next(key,value);
+ offset = ((MR_long)((Tuple)(value.data())).first()).get();
+ size = ((MR_long)((Tuple)(value.data())).second()).get();
+ this.source_number = source_number;
+ source_num_data = new MR_int(source_number);
+ index = -1;
+ }
+
+ public MRContainer createKey () {
+ return new MRContainer(null);
+ }
+
+ public MRContainer createValue () {
+ return new MRContainer(null);
+ }
+
+ public boolean next ( MRContainer key, MRContainer value ) throws IOException {
+ index++;
+ value.set(new Tuple(source_num_data,new MR_long(offset+index)));
+ key.set(new MR_long(index));
+ return index < size;
+ }
+
+ public long getPos () throws IOException { return index; }
+
+ public void close () throws IOException { reader.close(); }
+
+ public float getProgress () throws IOException {
+ return index / (float)size;
+ }
+
+ public void initialize ( InputSplit split, TaskAttemptContext context ) throws IOException { }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split, BSPJob job ) throws IOException {
+ Configuration conf = BSPPlan.getConfiguration(job);
+ String path = ((FileSplit)split).getPath().toString();
+ GeneratorDataSource ds = (GeneratorDataSource)DataSource.get(path,conf);
+ return new GeneratorRecordReader((FileSplit)split,ds.source_num,job);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/BSPMRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/BSPMRQLFileInputFormat.java b/bsp/src/main/java/org/apache/mrql/BSPMRQLFileInputFormat.java
new file mode 100644
index 0000000..71518cd
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/BSPMRQLFileInputFormat.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hama.bsp.*;
+import org.apache.hama.HamaConfiguration;
+
+
+/** A superclass for all MRQL FileInputFormats */
+abstract public class BSPMRQLFileInputFormat extends FileInputFormat<MRContainer,MRContainer> implements MRQLFileInputFormat {
+ public BSPMRQLFileInputFormat () {}
+
+ /** record reader for map-reduce */
+ abstract public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split, BSPJob job ) throws IOException;
+
+ /** materialize the input file into a memory Bag */
+ public Bag materialize ( final Path file ) throws IOException {
+ final BSPJob job = new BSPJob((HamaConfiguration)Plan.conf,MRQLFileInputFormat.class);
+ job.setInputPath(file);
+ final InputSplit[] splits = getSplits(job,1);
+ final RecordReader<MRContainer,MRContainer> rd = getRecordReader(splits[0],job);
+ return new Bag(new BagIterator () {
+ RecordReader<MRContainer,MRContainer> reader = rd;
+ MRContainer key = reader.createKey();
+ MRContainer value = reader.createKey();
+ int i = 0;
+ public boolean hasNext () {
+ try {
+ if (reader.next(key,value))
+ return true;
+ do {
+ if (++i >= splits.length)
+ return false;
+ reader.close();
+ reader = getRecordReader(splits[i],job);
+ } while (!reader.next(key,value));
+ return true;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from an intermediate result");
+ }
+ }
+ public MRData next () {
+ return value.data();
+ }
+ });
+ }
+
+ /** materialize the entire dataset into a Bag
+ * @param x the DataSet in HDFS to collect values from
+ * @param strip true if you want to stripout the source id (used in BSP sources)
+ * @return the Bag that contains the collected values
+ */
+ public final Bag collect ( final DataSet x, boolean strip ) throws Exception {
+ Bag res = new Bag();
+ for ( DataSource s: x.source )
+ if (s.to_be_merged)
+ res = res.union(Plan.merge(s));
+ else {
+ Path path = new Path(s.path);
+ final FileSystem fs = path.getFileSystem(Plan.conf);
+ final FileStatus[] ds
+ = fs.listStatus(path,
+ new PathFilter () {
+ public boolean accept ( Path path ) {
+ return !path.getName().startsWith("_");
+ }
+ });
+ Bag b = new Bag();
+ for ( FileStatus st: ds )
+ b = b.union(s.inputFormat.newInstance().materialize(st.getPath()));
+ if (strip) {
+ // remove source_num
+ final Iterator<MRData> iter = b.iterator();
+ b = new Bag(new BagIterator() {
+ public boolean hasNext () {
+ return iter.hasNext();
+ }
+ public MRData next () {
+ return ((Tuple)iter.next()).get(1);
+ }
+ });
+ };
+ res = res.union(b);
+ };
+ return res;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/bsp/src/main/java/org/apache/mrql/BSPParsedInputFormat.java
----------------------------------------------------------------------
diff --git a/bsp/src/main/java/org/apache/mrql/BSPParsedInputFormat.java b/bsp/src/main/java/org/apache/mrql/BSPParsedInputFormat.java
new file mode 100644
index 0000000..7da9a2e
--- /dev/null
+++ b/bsp/src/main/java/org/apache/mrql/BSPParsedInputFormat.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.Trees;
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hama.bsp.*;
+
+
+/** A FileInputFormat for text files (CVS, XML, JSON, ...) */
+final public class BSPParsedInputFormat extends BSPMRQLFileInputFormat {
+ public static class ParsedRecordReader implements RecordReader<MRContainer,MRContainer> {
+ final FSDataInputStream fsin;
+ final long start;
+ final long end;
+ final int source_number;
+ final MRData source_num_data;
+ Iterator<MRData> result;
+ Parser parser;
+
+ public ParsedRecordReader ( FileSplit split,
+ BSPJob job,
+ Class<? extends Parser> parser_class,
+ int source_number,
+ Trees args ) throws IOException {
+ Configuration conf = BSPPlan.getConfiguration(job);
+ start = split.getStart();
+ end = start + split.getLength();
+ Path file = split.getPath();
+ FileSystem fs = file.getFileSystem(conf);
+ fsin = fs.open(split.getPath());
+ try {
+ parser = parser_class.newInstance();
+ } catch (Exception ex) {
+ throw new Error("Unrecognized parser:"+parser_class);
+ };
+ this.source_number = source_number;
+ source_num_data = new MR_int(source_number);
+ parser.initialize(args);
+ parser.open(fsin,start,end);
+ result = null;
+ }
+
+ public MRContainer createKey () {
+ return new MRContainer();
+ }
+
+ public MRContainer createValue () {
+ return new MRContainer();
+ }
+
+ public synchronized boolean next ( MRContainer key, MRContainer value ) throws IOException {
+ while (result == null || !result.hasNext()) {
+ String s = parser.slice();
+ if (s == null)
+ return false;
+ result = parser.parse(s).iterator();
+ };
+ value.set(new Tuple(source_num_data,(MRData)result.next()));
+ key.set(new MR_long(fsin.getPos()));
+ return true;
+ }
+
+ public synchronized long getPos () throws IOException { return fsin.getPos(); }
+
+ public synchronized void close () throws IOException { fsin.close(); }
+
+ public float getProgress () throws IOException {
+ if (end == start)
+ return 0.0f;
+ else return Math.min(1.0f, (getPos() - start) / (float)(end - start));
+ }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ getRecordReader ( InputSplit split,
+ BSPJob job ) throws IOException {
+ Configuration conf = BSPPlan.getConfiguration(job);
+ String path = ((FileSplit)split).getPath().toString();
+ ParsedDataSource ds = (ParsedDataSource)DataSource.get(path,conf);
+ return new ParsedRecordReader((FileSplit)split,job,ds.parser,ds.source_num,(Trees)ds.args);
+ }
+}
[15/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/JoinOperation.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/JoinOperation.java b/mapreduce/src/main/java/org/apache/mrql/JoinOperation.java
new file mode 100644
index 0000000..0f962df
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/JoinOperation.java
@@ -0,0 +1,405 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.Enumeration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+
+
+/** The MapReduce2 physical operator (a reduce-side join) */
+final public class JoinOperation extends MapReducePlan {
+
+ /** Container for join input values. For reduce-side join, we concatenate
+ * 1 or 2 to the join key before we mix the right with the left tuples */
+ public final static class JoinKey implements Writable {
+ public byte tag; // 1 or 2
+ public MRData key;
+
+ JoinKey () {}
+ JoinKey ( byte t, MRData k ) { tag = t; key = k; }
+
+ public void write ( DataOutput out ) throws IOException {
+ out.writeByte(tag);
+ key.write(out);
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ tag = in.readByte();
+ key = MRContainer.read(in);
+ }
+
+ public String toString () {
+ return "<"+tag+":"+key+">";
+ }
+ }
+
+ public final static class MRContainerJoinPartitioner extends Partitioner<JoinKey,MRContainer> {
+ final public int getPartition ( JoinKey key, MRContainer value, int numPartitions ) {
+ return Math.abs(key.key.hashCode()) % numPartitions;
+ }
+ }
+
+ /** The sorting of the joined values uses ths join key for major order and tag for minor order */
+ public final static class MRContainerSortComparator implements RawComparator<JoinKey> {
+ int[] container_size;
+
+ public MRContainerSortComparator () {
+ container_size = new int[1];
+ }
+
+ final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
+ int cmp = MRContainer.compare(x,xs+1,xl-1,y,ys+1,yl-1,container_size);
+ return (cmp == 0) ? x[xs]-y[ys] : cmp;
+ }
+
+ final public int compare ( JoinKey x, JoinKey y ) {
+ int c = x.key.compareTo(y.key);
+ return (c == 0) ? x.tag-y.tag : c;
+ }
+ }
+
+ /** The grouping of the joined values is based on join key only */
+ public final static class MRContainerGroupingComparator implements RawComparator<JoinKey> {
+ int[] container_size;
+
+ public MRContainerGroupingComparator() {
+ container_size = new int[1];
+ }
+
+ final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
+ return MRContainer.compare(x,xs+1,xl-1,y,ys+1,yl-1,container_size);
+ }
+
+ final public int compare ( JoinKey x, JoinKey y ) {
+ return x.key.compareTo(y.key);
+ }
+ }
+
+ /** The left mapper for MapReduce2 */
+ private final static class MapperLeft extends Mapper<MRContainer,MRContainer,JoinKey,MRContainer> {
+ private static Function mx; // the left mapper function
+ private static JoinKey join_key = new JoinKey((byte)2,new MR_int(0));
+ private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(2));
+ private static MRContainer cvalue = new MRContainer(tvalue);
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ for ( MRData e: (Bag)mx.eval(value.data()) ) {
+ Tuple p = (Tuple)e;
+ join_key.key = p.first();
+ tvalue.set(1,p.second());
+ cvalue.set(tvalue);
+ context.write(join_key,cvalue);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.mapper.left"));
+ mx = functional_argument(conf,code);
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the left mapper plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ super.cleanup(context);
+ }
+ }
+
+ /** The right mapper for MapReduce2 */
+ private final static class MapperRight extends Mapper<MRContainer,MRContainer,JoinKey,MRContainer> {
+ private static Function my; // the right mapper function
+ private static JoinKey join_key = new JoinKey((byte)1,new MR_int(0));
+ private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(1));
+ private static MRContainer cvalue = new MRContainer(tvalue);
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ for ( MRData e: (Bag)my.eval(value.data()) ) {
+ Tuple p = (Tuple)e;
+ join_key.key = p.first();
+ tvalue.set(1,p.second());
+ cvalue.set(tvalue);
+ context.write(join_key,cvalue);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.mapper.right"));
+ my = functional_argument(conf,code);
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the right mapper plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ super.cleanup(context);
+ }
+ }
+
+ /** The reducer for MapReduce2 */
+ private static class JoinReducer extends Reducer<JoinKey,MRContainer,MRContainer,MRContainer> {
+ private static String counter; // a Hadoop user-defined counter used in the repeat operation
+ private static Function combine_fnc; // the combiner function
+ private static Function reduce_fnc; // the reduce function
+ private static Bag left = new Bag(); // a cached bag of input fragments from left input
+ private static Bag right = new Bag(); // a cached bag of input fragments from right input
+ private static Function acc_fnc; // aggregator
+ private static MRData result; // aggregation result
+ private static Hashtable<MRData,MRData> hashTable; // in-mapper combiner
+ private static int index;
+ private static Tuple tkey = new Tuple(2);
+ private static Bag tbag = new Bag(2);
+ private static boolean streamed = false;
+ private static Tuple pair = new Tuple(2);
+ private static MRContainer ckey = new MRContainer(new MR_int(0));
+ private static MRContainer cvalue = new MRContainer(new MR_int(0));
+ private static MRContainer container = new MRContainer(new MR_int(0));
+
+ private void write ( MRContainer key, MRData value, Context context )
+ throws IOException, InterruptedException {
+ if (result != null) { // aggregation
+ pair.set(0,result);
+ pair.set(1,value);
+ result = acc_fnc.eval(pair);
+ } else if (hashTable != null) {
+ MRData k = ((Tuple)value).get(0);
+ MRData v = ((Tuple)value).get(1);
+ MRData old = hashTable.get(k);
+ if (old == null) {
+ if (index++ == Config.map_cache_size)
+ flush_table(context);
+ hashTable.put(k,v);
+ } else {
+ tkey.set(0,key.data());
+ tbag.clear();
+ tbag.add_element(v).add_element(old);
+ tkey.set(1,tbag);
+ for ( MRData x: (Bag)combine_fnc.eval(tkey) )
+ hashTable.put(k,x); // normally, done once
+ }
+ } else if (counter.equals("-")) {
+ container.set(value);
+ context.write(key,container);
+ } else { // increment the repetition counter if the repeat condition is true
+ Tuple t = (Tuple)value;
+ if (((MR_bool)t.second()).get())
+ context.getCounter("mrql",counter).increment(1);
+ container.set(t.first());
+ context.write(key,container);
+ }
+ }
+
+ protected static void flush_table ( Context context ) throws IOException, InterruptedException {
+ Enumeration<MRData> en = hashTable.keys();
+ while (en.hasMoreElements()) {
+ MRData key = en.nextElement();
+ ckey.set(key);
+ MRData value = hashTable.get(key);
+ cvalue.set(value);
+ if (value != null)
+ context.write(ckey,new MRContainer(new Tuple(key,value)));
+ };
+ index = 0;
+ hashTable.clear();
+ }
+
+ @Override
+ public void reduce ( JoinKey key, Iterable<MRContainer> values, Context context )
+ throws IOException, InterruptedException {
+ if (!streamed) {
+ left.clear();
+ right.clear();
+ for ( MRContainer val: values ) {
+ Tuple p = (Tuple)val.data();
+ if (((MR_byte)p.first()).get() == 1)
+ right.add(p.second());
+ else left.add(p.second());
+ };
+ } else { // the left input is processed lazily (as a stream-based bag)
+ right.clear();
+ Tuple p = null;
+ final Iterator<MRContainer> i = values.iterator();
+ while (i.hasNext()) {
+ p = (Tuple)i.next().data();
+ if (((MR_byte)p.first()).get() == 2)
+ break;
+ right.add(p.second());
+ p = null;
+ };
+ final Tuple data = p;
+ left = new Bag(new BagIterator () {
+ boolean first_time = data != null;
+ public boolean hasNext () {
+ return first_time || i.hasNext();
+ }
+ public MRData next () {
+ if (!first_time) {
+ Tuple t = (Tuple)i.next().data();
+ assert(((MR_byte)(t.first())).get() == 2);
+ return t.second();
+ };
+ first_time = false;
+ return data.second();
+ }
+ });
+ };
+ pair.set(0,left);
+ pair.set(1,right);
+ for ( MRData e: (Bag)reduce_fnc.eval(pair) ) {
+ ckey.set(key.key);
+ write(ckey,e,context);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.reducer"));
+ reduce_fnc = functional_argument(conf,code);
+ streamed = PlanGeneration.streamed_MapReduce2_reducer(code);
+ if (conf.get("mrql.zero") != null) {
+ code = Tree.parse(conf.get("mrql.zero"));
+ result = Interpreter.evalE(code);
+ code = Tree.parse(conf.get("mrql.accumulator"));
+ acc_fnc = functional_argument(conf,code);
+ } else result = null;
+ counter = conf.get("mrql.counter");
+ code = Tree.parse(conf.get("mrql.combiner"));
+ hashTable = null;
+ if (code != null && !code.equals(new VariableLeaf("null"))) {
+ combine_fnc = functional_argument(conf,code);
+ hashTable = new Hashtable<MRData,MRData>(Config.map_cache_size);
+ index = 0;
+ }
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the reducer plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (result != null) // emit the result of aggregation
+ context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
+ if (hashTable != null)
+ flush_table(context);
+ hashTable = null; // garbage-collect it
+ super.cleanup(context);
+ }
+ }
+
+ /** The MapReduce2 physical operator (a reduce-side join)
+ * @param mx left mapper function
+ * @param my right mapper function
+ * @param combine_fnc optional in-mapper combiner function
+ * @param reduce_fnc reducer function
+ * @param acc_fnc optional accumulator function
+ * @param zero optional the zero value for the accumulator
+ * @param X left data set
+ * @param Y right data set
+ * @param num_reduces number of reducers
+ * @param stop_counter optional counter used in repeat operation
+ * @param orderp does the result need to be ordered?
+ * @return a new data source that contains the result
+ */
+ public final static DataSet mapReduce2 ( Tree mx, // left mapper function
+ Tree my, // right mapper function
+ Tree combine_fnc, // optional in-mapper combiner function
+ Tree reduce_fnc, // reducer function
+ Tree acc_fnc, // optional accumulator function
+ Tree zero, // optional the zero value for the accumulator
+ DataSet X, // left data set
+ DataSet Y, // right data set
+ int num_reduces, // number of reducers
+ String stop_counter, // optional counter used in repeat operation
+ boolean orderp ) // does the result need to be ordered?
+ throws Exception {
+ String newpath = new_path(conf);
+ conf.set("mrql.mapper.left",mx.toString());
+ conf.set("mrql.mapper.right",my.toString());
+ if (combine_fnc != null)
+ conf.set("mrql.combiner",combine_fnc.toString());
+ conf.set("mrql.reducer",reduce_fnc.toString());
+ if (zero != null) {
+ conf.set("mrql.accumulator",acc_fnc.toString());
+ conf.set("mrql.zero",zero.toString());
+ // the in-mapper combiner likes large data splits
+ conf.set("mapred.min.split.size","268435456"); // 256 MBs
+ } else conf.set("mrql.zero","");
+ conf.set("mrql.counter",stop_counter);
+ Job job = new Job(conf,newpath);
+ distribute_compiled_arguments(job.getConfiguration());
+ job.setMapOutputKeyClass(JoinKey.class);
+ job.setJarByClass(MapReducePlan.class);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setPartitionerClass(MRContainerJoinPartitioner.class);
+ job.setSortComparatorClass(MRContainerSortComparator.class);
+ job.setGroupingComparatorClass(MRContainerGroupingComparator.class);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ FileOutputFormat.setOutputPath(job,new Path(newpath));
+ for (DataSource p: X.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperLeft.class);
+ for (DataSource p: Y.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperRight.class);
+ if (Config.trace && PlanGeneration.streamed_MapReduce2_reducer(reduce_fnc))
+ System.err.println("*** Streamed MapReduce2 reducer");
+ job.setReducerClass(JoinReducer.class);
+ if (num_reduces > 0)
+ job.setNumReduceTasks(num_reduces);
+ job.waitForCompletion(true);
+ long c = (stop_counter.equals("-")) ? 0
+ : job.getCounters().findCounter("mrql",stop_counter).getValue();
+ DataSource s = new BinaryDataSource(newpath,conf);
+ s.to_be_merged = orderp;
+ return new DataSet(s,c,outputRecords(job));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapJoinOperation.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapJoinOperation.java b/mapreduce/src/main/java/org/apache/mrql/MapJoinOperation.java
new file mode 100644
index 0000000..3ac4efa
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapJoinOperation.java
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.net.URI;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.Enumeration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.filecache.DistributedCache;
+
+
+/** The fragment-replicate join (map-side join) physical operator */
+final public class MapJoinOperation extends MapReducePlan {
+
+ /** the mapper of the MapJoin */
+ private final static class mapJoinMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
+ private static String counter; // a Hadoop user-defined counter used in the repeat operation
+ private static Function reduce_fnc; // the reduce function
+ private static Function probe_map_fnc;
+ private static Hashtable<MRData,Bag> built_table;
+ private static Function acc_fnc; // aggregator
+ private static MRData result; // aggregation result
+ private static Tuple pair = new Tuple(2);
+ private static MRContainer container = new MRContainer(new MR_int(0));
+ private static Bag empty_bag = new Bag();
+ private static boolean mapJoinReduce = false;
+
+ private void write ( MRContainer key, MRData value, Context context )
+ throws IOException, InterruptedException {
+ if (result != null) { // aggregation
+ pair.set(0,result);
+ pair.set(1,value);
+ result = acc_fnc.eval(pair);
+ } else if (counter.equals("-")) {
+ container.set(value);
+ context.write(key,container);
+ } else { // increment the repetition counter if the repeat condition is true
+ Tuple t = (Tuple)value;
+ if (((MR_bool)t.second()).get())
+ context.getCounter("mrql",counter).increment(1);
+ container.set(t.first());
+ context.write(key,container);
+ }
+ }
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ for ( MRData e: (Bag)probe_map_fnc.eval(value.data()) ) {
+ Tuple p = (Tuple)e;
+ MRData pd = built_table.get(p.first());
+ if (pd == null)
+ pd = empty_bag;
+ pair.set(0,p.second());
+ pair.set(1,pd);
+ for ( MRData v: (Bag)reduce_fnc.eval(pair) )
+ write(key,v,context);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ if (conf.get("mrql.mapJoinReduce") != null)
+ mapJoinReduce = true;
+ Tree code = Tree.parse(conf.get("mrql.inMap.reducer"));
+ reduce_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.probe_mapper"));
+ probe_map_fnc = functional_argument(conf,code);
+ if (!mapJoinReduce && conf.get("mrql.zero") != null) {
+ code = Tree.parse(conf.get("mrql.zero"));
+ result = Interpreter.evalE(code);
+ code = Tree.parse(conf.get("mrql.accumulator"));
+ acc_fnc = functional_argument(conf,code);
+ } else result = null;
+ counter = conf.get("mrql.counter");
+ built_table = new Hashtable<MRData,Bag>(Config.map_cache_size);
+ Bag res = new Bag();
+ URI[] uris = DistributedCache.getCacheFiles(conf);
+ Path[] local_paths = DistributedCache.getLocalCacheFiles(conf);
+ final FileSystem fs = FileSystem.getLocal(conf);
+ final Configuration fconf = conf;
+ for ( int i = 0; i < local_paths.length; i++ ) {
+ // hadoop 0.20.2 distributed cache doesn't work in stand-alone
+ final Path path = (conf.get("mapred.job.tracker").equals("local"))
+ ? new Path(uris[i].toString())
+ : local_paths[i];
+ if (path.getName().endsWith(".jar"))
+ continue;
+ res = res.union(new Bag(new BagIterator () {
+ final SequenceFile.Reader reader = new SequenceFile.Reader(fs,path,fconf);
+ final MRContainer key = new MRContainer(new MR_int(0));
+ final MRContainer value = new MRContainer(new MR_int(0));
+ public boolean hasNext () {
+ try {
+ boolean done = reader.next(key,value);
+ if (!done)
+ reader.close();
+ return done;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from distributed cache");
+ }
+ }
+ public MRData next () {
+ return value.data();
+ }
+ }));
+ };
+ for ( MRData e: res ) {
+ Tuple p = (Tuple)e;
+ Bag entries = built_table.get(p.first());
+ built_table.put(p.first(),
+ (entries == null)
+ ? (new Bag(p.second()))
+ : entries.add_element(p.second()));
+ }
+ } catch (Exception e) {
+ throw new Error("Cannot setup the mapJoin: "+e);
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (result != null) // emit the result of aggregation
+ context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
+ built_table = null; // garbage-collect it
+ super.cleanup(context);
+ }
+ }
+
+ /** The fragment-replicate join (map-side join) physical operator
+ * @param probe_map_fnc left mapper function
+ * @param built_map_fnc right mapper function
+ * @param reduce_fnc reducer function
+ * @param acc_fnc optional accumulator function
+ * @param zero optional the zero value for the accumulator
+ * @param probe_dataset the map source
+ * @param built_dataset stored in distributed cache
+ * @param stop_counter optional counter used in repeat operation
+ * @return a new data source that contains the result
+ */
+ public final static DataSet mapJoin ( Tree probe_map_fnc, // left mapper function
+ Tree built_map_fnc, // right mapper function
+ Tree reduce_fnc, // reducer function
+ Tree acc_fnc, // optional accumulator function
+ Tree zero, // optional the zero value for the accumulator
+ DataSet probe_dataset, // the map source
+ DataSet built_dataset, // stored in distributed cache
+ String stop_counter ) // optional counter used in repeat operation
+ throws Exception {
+ DataSet ds = MapOperation.cMap(built_map_fnc,null,null,built_dataset,"-");
+ String newpath = new_path(conf);
+ conf.set("mrql.inMap.reducer",reduce_fnc.toString());
+ conf.set("mrql.probe_mapper",probe_map_fnc.toString());
+ conf.set("mrql.counter",stop_counter);
+ if (zero != null) {
+ conf.set("mrql.accumulator",acc_fnc.toString());
+ conf.set("mrql.zero",zero.toString());
+ conf.set("mapred.min.split.size","268435456");
+ } else conf.set("mrql.zero","");
+ Job job = new Job(conf,newpath);
+ distribute_compiled_arguments(job.getConfiguration());
+ job.setJarByClass(MapReducePlan.class);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ PathFilter pf = new PathFilter () { public boolean accept ( Path path ) {
+ return !path.getName().startsWith("_");
+ } };
+ for (DataSource p: ds.source) { // distribute the built dataset
+ Path path = new Path(p.path);
+ for ( FileStatus s: path.getFileSystem(conf).listStatus(path,pf) )
+ DistributedCache.addCacheFile(s.getPath().toUri(),job.getConfiguration());
+ };
+ for (DataSource p: probe_dataset.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,mapJoinMapper.class);
+ FileOutputFormat.setOutputPath(job,new Path(newpath));
+ job.setNumReduceTasks(0);
+ job.waitForCompletion(true);
+ long c = (stop_counter.equals("-")) ? 0
+ : job.getCounters().findCounter("mrql",stop_counter).getValue();
+ return new DataSet(new BinaryDataSource(newpath,conf),c,outputRecords(job));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapOperation.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapOperation.java b/mapreduce/src/main/java/org/apache/mrql/MapOperation.java
new file mode 100644
index 0000000..b5c9a8f
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapOperation.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+
+/** The MapReduce operation that use a map stage only */
+final public class MapOperation extends MapReducePlan {
+
+ /** The mapper of Map */
+ private final static class cMapMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
+ private static String counter; // a Hadoop user-defined counter used in the repeat operation
+ private static Function map_fnc; // the mapper function
+ private static Function acc_fnc; // aggregator
+ private static MRData result; // aggregation result
+ private static Tuple pair = new Tuple(2);
+ private static MRContainer container = new MRContainer(new MR_int(0));
+
+ private void write ( MRContainer key, MRData value, Context context )
+ throws IOException, InterruptedException {
+ if (result != null) { // aggregation
+ pair.set(0,result);
+ pair.set(1,value);
+ result = acc_fnc.eval(pair);
+ } else if (counter.equals("-")) {
+ container.set(value);
+ context.write(key,container);
+ } else { // increment the repetition counter if the repeat condition is true
+ Tuple t = (Tuple)value;
+ if (((MR_bool)t.second()).get())
+ context.getCounter("mrql",counter).increment(1);
+ container.set(t.first());
+ context.write(key,container);
+ }
+ }
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ for ( MRData e: (Bag)map_fnc.eval(value.data()) )
+ write(key,e,context);
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.mapper"));
+ map_fnc = functional_argument(conf,code);
+ if (conf.get("mrql.zero") != null) {
+ code = Tree.parse(conf.get("mrql.zero"));
+ result = Interpreter.evalE(code);
+ code = Tree.parse(conf.get("mrql.accumulator"));
+ acc_fnc = functional_argument(conf,code);
+ } else result = null;
+ counter = conf.get("mrql.counter");
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the mapper plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (result != null) // emit the result of aggregation
+ context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
+ super.cleanup(context);
+ }
+ }
+
+ /** The cMap physical operator
+ * @param map_fnc mapper function
+ * @param acc_fnc optional accumulator function
+ * @param zero optional the zero value for the accumulator
+ * @param source input data source
+ * @param stop_counter optional counter used in repeat operation
+ * @return a new data source that contains the result
+ */
+ public final static DataSet cMap ( Tree map_fnc, // mapper function
+ Tree acc_fnc, // optional accumulator function
+ Tree zero, // optional the zero value for the accumulator
+ DataSet source, // input data source
+ String stop_counter ) // optional counter used in repeat operation
+ throws Exception {
+ String newpath = new_path(conf);
+ conf.set("mrql.mapper",map_fnc.toString());
+ conf.set("mrql.counter",stop_counter);
+ if (zero != null) {
+ conf.set("mrql.accumulator",acc_fnc.toString());
+ conf.set("mrql.zero",zero.toString());
+ conf.set("mapred.min.split.size","268435456");
+ } else conf.set("mrql.zero","");
+ Job job = new Job(conf,newpath);
+ distribute_compiled_arguments(job.getConfiguration());
+ job.setJarByClass(MapReducePlan.class);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ for (DataSource p: source.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,cMapMapper.class);
+ FileOutputFormat.setOutputPath(job,new Path(newpath));
+ job.setNumReduceTasks(0);
+ job.waitForCompletion(true);
+ long c = (stop_counter.equals("-")) ? 0
+ : job.getCounters().findCounter("mrql",stop_counter).getValue();
+ return new DataSet(new BinaryDataSource(newpath,conf),c,outputRecords(job));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReduceBinaryInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReduceBinaryInputFormat.java b/mapreduce/src/main/java/org/apache/mrql/MapReduceBinaryInputFormat.java
new file mode 100644
index 0000000..36ad094
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReduceBinaryInputFormat.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+
+
+/** Input format for Apache Hadoop sequence files */
+final public class MapReduceBinaryInputFormat extends MapReduceMRQLFileInputFormat {
+ final static SequenceFileInputFormat<MRContainer,MRContainer> inputFormat
+ = new SequenceFileInputFormat<MRContainer,MRContainer>();
+
+ public RecordReader<MRContainer,MRContainer>
+ createRecordReader ( InputSplit split,
+ TaskAttemptContext context ) throws IOException, InterruptedException {
+ return inputFormat.createRecordReader(split,context);
+ }
+
+ /** collect the data from multiple sequence files at the path directory into a Bag
+ * @param path the path directory
+ * @return a Bag that contains all data
+ */
+ public Bag materialize ( final Path path ) throws IOException {
+ final FileSystem fs = path.getFileSystem(Plan.conf);
+ final FileStatus[] ds
+ = fs.listStatus(path,
+ new PathFilter () {
+ public boolean accept ( Path path ) {
+ return !path.getName().startsWith("_");
+ }
+ });
+ if (ds.length > 0)
+ return new Bag(new BagIterator () {
+ SequenceFile.Reader reader = new SequenceFile.Reader(fs,ds[0].getPath(),Plan.conf);
+ MRContainer key = new MRContainer(new MR_int(0));
+ MRContainer value = new MRContainer(new MR_int(0));
+ int i = 1;
+ public boolean hasNext () {
+ try {
+ if (reader.next(key,value))
+ return true;
+ do {
+ if (i >= ds.length)
+ return false;
+ reader.close();
+ reader = new SequenceFile.Reader(fs,ds[i++].getPath(),Plan.conf);
+ } while (!reader.next(key,value));
+ return true;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from an intermediate result");
+ }
+ }
+ public MRData next () {
+ return value.data();
+ }
+ });
+ return new Bag();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReduceEvaluator.gen
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReduceEvaluator.gen b/mapreduce/src/main/java/org/apache/mrql/MapReduceEvaluator.gen
new file mode 100644
index 0000000..e733715
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReduceEvaluator.gen
@@ -0,0 +1,296 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java_cup.runtime.*;
+import org.apache.mrql.gen.*;
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** Evaluates physical plans in Apache Hadoop MapReduce mode */
+final public class MapReduceEvaluator extends Evaluator {
+
+ /** initialize the MapReduce evaluator */
+ final public void init ( Configuration conf ) {
+ Config.map_reduce_mode = true;
+ if (Config.hadoop_mode)
+ if (Config.local_mode) {
+ conf.set("mapred.job.tracker","local");
+ conf.set("fs.default.name","file:///");
+ } else {
+ conf.set("mapred.job.tracker",System.getenv("MAPRED_JOB_TRACKER"));
+ conf.set("fs.default.name",System.getenv("FS_DEFAULT_NAME"));
+ }
+ }
+
+ /** shutdown the MapReduce evaluator */
+ final public void shutdown ( Configuration conf ) {
+ }
+
+ /** initialize the query evaluation */
+ final public void initialize_query () {
+ }
+
+ /** create a new evaluation configuration */
+ final public Configuration new_configuration () {
+ return new Configuration();
+ }
+
+ /** return the FileInputFormat for parsed files (CSV, XML, JSON, etc) */
+ final public Class<? extends MRQLFileInputFormat> parsedInputFormat () {
+ return MapReduceParsedInputFormat.class;
+ }
+
+ /** return the FileInputFormat for binary files */
+ final public Class<? extends MRQLFileInputFormat> binaryInputFormat () {
+ return MapReduceBinaryInputFormat.class;
+ }
+
+ /** return the FileInputFormat for data generator files */
+ final public Class<? extends MRQLFileInputFormat> generatorInputFormat () {
+ return MapReduceGeneratorInputFormat.class;
+ }
+
+ /** The Aggregate physical operator
+ * @param acc_fnc the accumulator function from (T,T) to T
+ * @param zero the zero element of type T
+ * @param plan the plan that constructs the dataset that contains the bag of values {T}
+ * @param env contains bindings fro variables to values (MRData)
+ * @return the aggregation result of type T
+ */
+ final public MRData aggregate ( Tree acc_fnc,
+ Tree zero,
+ Tree plan,
+ Environment env ) throws Exception {
+ return MapReducePlan.aggregate(acc_fnc,zero,eval(plan,env,"-"));
+ }
+
+ /** Evaluate a loop a fixed # of times */
+ final public Tuple loop ( Tree e, Environment env ) throws Exception {
+ match e {
+ case Loop(lambda(tuple(...vs),tuple(...bs)),tuple(...ss),`num):
+ int limit = ((MR_int)evalE(num,env)).get();
+ MR_dataset[] s = new MR_dataset[vs.length()];
+ for ( int i = 0; i < vs.length(); i++ )
+ s[i] = new MR_dataset(eval(ss.nth(i),env,"-"));
+ for ( int n = 0; n < limit; n++ ) {
+ Environment nenv = env;
+ for ( int i = 0; i < vs.length(); i ++ )
+ nenv = new Environment(vs.nth(i).toString(),s[i],nenv);
+ for ( int i = 0; i < vs.length(); i ++ )
+ s[i] = new MR_dataset(eval(bs.nth(i),nenv,"-"));
+ };
+ return new Tuple(s);
+ };
+ throw new Error("Wrong Loop format");
+ }
+
+ /** Evaluate an MRQL physical plan using Hadoop and print tracing info
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a DataSet (stored in HDFS)
+ */
+ final public DataSet eval ( final Tree e,
+ final Environment env,
+ final String counter ) {
+ if (Config.trace_execution) {
+ tab_count += 3;
+ System.out.println(tabs(tab_count)+print_query(e));
+ };
+ DataSet res = evalD(e,env,counter);
+ if (Config.trace_execution)
+ try {
+ System.out.println(tabs(tab_count)
+ +"-> "+res.take(Config.max_bag_size_print));
+ tab_count -= 3;
+ } catch (Exception ex) {
+ throw new Error("Cannot collect the operator output: "+e);
+ };
+ return res;
+ }
+
+ /** Evaluate MRQL physical operators using Hadoop
+ * @param e the physical plan
+ * @param env contains bindings fro variables to values (MRData)
+ * @return a DataSet (stored in HDFS)
+ */
+ final DataSet evalD ( final Tree e,
+ final Environment env,
+ final String counter ) {
+ try {
+ match e {
+ case cMap(`f,`s):
+ return MapOperation.cMap(closure(f,env),null,null,eval(s,env,"-"),counter);
+ case AggregateMap(`f,`acc,`zero,`s):
+ return MapOperation.cMap(closure(f,env),closure(acc,env),
+ (zero.equals(#<null>))?null:zero,
+ eval(s,env,"-"),counter);
+ case MapReduce(`m,`r,`s,`o):
+ return MapReduceOperation.mapReduce(closure(m,env),#<null>,closure(r,env),
+ null,null,
+ eval(s,env,"-"),
+ Config.nodes,counter,
+ o.equals(#<true>));
+ case MapAggregateReduce(`m,`r,`acc,`zero,`s,`o):
+ return MapReduceOperation.mapReduce(closure(m,env),null,closure(r,env),
+ closure(acc,env),
+ (zero.equals(#<null>))?null:zero,
+ eval(s,env,"-"),
+ Config.nodes,counter,
+ o.equals(#<true>));
+ case MapCombineReduce(`m,`c,`r,`s,`o):
+ return MapReduceOperation.mapReduce(closure(m,env),closure(c,env),closure(r,env),
+ null,null,
+ eval(s,env,"-"),
+ Config.nodes,counter,
+ o.equals(#<true>));
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return CrossProductOperation.crossProduct(closure(mx,env),closure(my,env),closure(r,env),
+ null,null,
+ eval(x,env,"-"),
+ eval(y,env,"-"),
+ counter);
+ case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
+ return CrossProductOperation.crossProduct(closure(mx,env),closure(my,env),closure(r,env),
+ closure(acc,env),
+ (zero.equals(#<null>))?null:zero,
+ eval(x,env,"-"),
+ eval(y,env,"-"),
+ counter);
+ case MapReduce2(`mx,`my,`r,`x,`y,`o):
+ return eval(#<MapAggregateReduce2(`mx,`my,`r,null,null,`x,`y,`o)>,env,counter);
+ case MapCombineReduce2(`mx,`my,`c,`r,`x,`y,`o):
+ return JoinOperation.mapReduce2(closure(mx,env),closure(my,env),
+ closure(c,env),
+ closure(r,env),
+ null,null,
+ eval(x,env,"-"),
+ eval(y,env,"-"),
+ Config.nodes,counter,
+ o.equals(#<true>));
+ case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,`o):
+ return JoinOperation.mapReduce2(closure(mx,env),closure(my,env),
+ null,
+ closure(r,env),closure(acc,env),
+ (zero.equals(#<null>))?null:zero,
+ eval(x,env,"-"),
+ eval(y,env,"-"),
+ Config.nodes,counter,
+ o.equals(#<true>));
+ case GroupByJoin(`kx,`ky,`gx,`gy,`mp,`c,`r,`x,`y,`o):
+ DataSet X = eval(x,env,"-");
+ DataSet Y = eval(y,env,"-");
+ int n = (int)Math.floor(Math.sqrt(4.0*Config.nodes));
+ int m = n;
+ if (Config.trace)
+ System.out.println("*** Using a groupBy join on a "+n+"*"+m+" grid of partitions");
+ return GroupByJoinPlan.groupByJoin(closure(kx,env),closure(ky,env),
+ closure(gx,env),closure(gy,env),
+ closure(mp,env),closure(c,env),closure(r,env),
+ X,Y,
+ Config.nodes,
+ n,m,counter);
+ case MapJoin(`mx,`my,`r,`x,`y):
+ return MapJoinOperation.mapJoin(closure(mx,env),closure(my,env),closure(r,env),
+ null,null,
+ eval(x,env,"-"),
+ eval(y,env,"-"),
+ counter);
+ case MapAggregateJoin(`mx,`my,`r,`acc,`zero,`x,`y):
+ return MapJoinOperation.mapJoin(closure(mx,env),closure(my,env),closure(r,env),
+ closure(acc,env),
+ (zero.equals(#<null>))?null:zero,
+ eval(x,env,"-"),
+ eval(y,env,"-"),
+ counter);
+ case BinarySource(`file,_):
+ return Plan.binarySource(file.stringValue());
+ case ParsedSource(`parser,`file,...args):
+ Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ return Plan.parsedSource(p,((MR_string)evalE(file,env)).get(),args);
+ case Merge(`x,`y):
+ return Plan.merge(eval(x,env,"-"),eval(y,env,"-"));
+ case Repeat(lambda(`v,`b),`s,`n):
+ final String nm = v.toString();
+ final Tree body = b;
+ Function loop_fnc = new Function () {
+ public MRData eval ( MRData s ) {
+ return new MR_dataset(evaluator.eval(body,new Environment(nm,s,env),nm));
+ }; };
+ return MapReducePlan.repeat(loop_fnc,eval(s,env,"-"),((MR_int)evalE(n,env)).get());
+ case Closure(lambda(`v,`b),`s,`n):
+ final String nm = v.toString();
+ final Tree body = b;
+ Function loop_fnc = new Function () {
+ public MRData eval ( MRData s ) {
+ return new MR_dataset(evaluator.eval(body,new Environment(nm,s,env),"-"));
+ }; };
+ return MapReducePlan.closure(loop_fnc,eval(s,env,"-"),((MR_int)evalE(n,env)).get());
+ case Generator(`min,`max,`size):
+ return Plan.generator(((MR_long)evalE(min,env)).get(),
+ ((MR_long)evalE(max,env)).get(),
+ ((MR_long)evalE(size,env)).get());
+ case let(`v,`u,`body):
+ return eval(body,new Environment(v.toString(),evalE(u,env),env),"-");
+ case Let(`v,`u,`body):
+ return eval(body,new Environment(v.toString(),new MR_dataset(eval(u,env,"-")),env),"-");
+ case If(`c,`x,`y):
+ if (((MR_bool)evalE(c,env)).get())
+ return eval(x,env,"-");
+ else return eval(y,env,"-");
+ case apply(`f,`arg):
+ if (!f.is_variable())
+ return ((MR_dataset)evalF(f,env).eval(evalE(arg))).dataset();
+ MRData fnc = variable_lookup(f.toString(),global_env);
+ if (fnc == null)
+ throw new Error("Unknown function: "+f);
+ MRData t = evalE(arg,env);
+ if (!(t instanceof Tuple))
+ throw new Error("Expected a tuple in function application: "+t);
+ return ((MR_dataset)((Lambda)fnc).lambda().eval(t)).dataset();
+ case `v:
+ if (!v.is_variable())
+ fail;
+ MRData x = variable_lookup(v.toString(),env);
+ if (x != null)
+ if (x instanceof MR_dataset)
+ return ((MR_dataset)x).dataset();
+ x = variable_lookup(v.toString(),global_env);
+ if (x != null)
+ if (x instanceof MR_dataset)
+ return ((MR_dataset)x).dataset();
+ throw new Error("Variable "+v+" is not bound");
+ };
+ throw new Error("Cannot evaluate the map-reduce plan: "+e);
+ } catch (Error msg) {
+ if (!Config.trace)
+ throw new Error(msg.getMessage());
+ System.err.println(msg.getMessage());
+ throw new Error("Evaluation error in: "+print_query(e));
+ } catch (Exception ex) {
+ System.err.println(ex.getMessage());
+ ex.printStackTrace();
+ throw new Error("Evaluation error in: "+print_query(e));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReduceGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReduceGeneratorInputFormat.java b/mapreduce/src/main/java/org/apache/mrql/MapReduceGeneratorInputFormat.java
new file mode 100644
index 0000000..9a8ec55
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReduceGeneratorInputFormat.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+
+/** the FileInputFormat for data generators: it creates HDFS files, where each file contains
+ * an (offset,size) pair that generates the range of values [offset,offset+size] */
+final public class MapReduceGeneratorInputFormat extends MapReduceMRQLFileInputFormat {
+ public static class GeneratorRecordReader extends RecordReader<MRContainer,MRContainer> {
+ final long offset;
+ final long size;
+ long index;
+ SequenceFile.Reader reader;
+
+ public GeneratorRecordReader ( FileSplit split,
+ TaskAttemptContext context ) throws IOException {
+ Configuration conf = context.getConfiguration();
+ Path path = split.getPath();
+ FileSystem fs = path.getFileSystem(conf);
+ reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
+ MRContainer key = new MRContainer();
+ MRContainer value = new MRContainer();
+ reader.next(key,value);
+ offset = ((MR_long)((Tuple)(value.data())).first()).get();
+ size = ((MR_long)((Tuple)(value.data())).second()).get();
+ index = -1;
+ }
+
+ public boolean nextKeyValue () throws IOException {
+ index++;
+ return index < size;
+ }
+
+ public MRContainer getCurrentKey () throws IOException {
+ return new MRContainer(new MR_long(index));
+ }
+
+ public MRContainer getCurrentValue () throws IOException {
+ return new MRContainer(new MR_long(offset+index));
+ }
+
+ public void close () throws IOException { reader.close(); }
+
+ public float getProgress () throws IOException {
+ return index / (float)size;
+ }
+
+ public void initialize ( InputSplit split, TaskAttemptContext context ) throws IOException { }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ createRecordReader ( InputSplit split, TaskAttemptContext context ) throws IOException {
+ return new GeneratorRecordReader((FileSplit)split,context);
+ }
+
+ /** Insert all results from the generators stored in path into a Bag.
+ * The Bag is lazily constructed.
+ * @param path the path directory that contains the generator data (offset,size)
+ * @return a Bag that contains all data
+ */
+ public Bag materialize ( final Path path ) throws IOException {
+ Configuration conf = Plan.conf;
+ FileSystem fs = path.getFileSystem(conf);
+ final SequenceFile.Reader reader = new SequenceFile.Reader(path.getFileSystem(conf),path,conf);
+ final MRContainer key = new MRContainer();
+ final MRContainer value = new MRContainer();
+ return new Bag(new BagIterator () {
+ long offset = 0;
+ long size = 0;
+ long i = 0;
+ public boolean hasNext () {
+ if (++i >= offset+size)
+ try {
+ if (!reader.next(key,value))
+ return false;
+ offset = ((MR_long)((Tuple)(value.data())).first()).get();
+ size = ((MR_long)((Tuple)(value.data())).second()).get();
+ i = offset;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from a generator");
+ };
+ return true;
+ }
+ public MRData next () {
+ return new MR_long(i);
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReduceMRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReduceMRQLFileInputFormat.java b/mapreduce/src/main/java/org/apache/mrql/MapReduceMRQLFileInputFormat.java
new file mode 100644
index 0000000..d0760b4
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReduceMRQLFileInputFormat.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+
+/** A superclass for all MRQL FileInputFormats */
+abstract public class MapReduceMRQLFileInputFormat extends FileInputFormat<MRContainer,MRContainer> implements MRQLFileInputFormat {
+ public MapReduceMRQLFileInputFormat () {}
+
+ /** record reader for map-reduce */
+ abstract public RecordReader<MRContainer,MRContainer>
+ createRecordReader ( InputSplit split,
+ TaskAttemptContext context ) throws IOException, InterruptedException;
+
+ /** materialize the input file into a memory Bag */
+ abstract public Bag materialize ( final Path path ) throws IOException;
+
+ /** materialize the entire dataset into a Bag
+ * @param x the DataSet in HDFS to collect values from
+ * @param strip is not used in MapReduce mode
+ * @return the Bag that contains the collected values
+ */
+ public final Bag collect ( final DataSet x, boolean strip ) throws Exception {
+ Bag res = new Bag();
+ for ( DataSource s: x.source )
+ if (s.to_be_merged)
+ res = res.union(Plan.merge(s));
+ else res = res.union(s.inputFormat.newInstance().materialize(new Path(s.path)));
+ return res;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReduceOperation.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReduceOperation.java b/mapreduce/src/main/java/org/apache/mrql/MapReduceOperation.java
new file mode 100644
index 0000000..39864b3
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReduceOperation.java
@@ -0,0 +1,273 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.Enumeration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+
+
+/** The MapReduce operation that uses an in-mapper combiner to partially reduce groups during mapping */
+final public class MapReduceOperation extends MapReducePlan {
+
+ public final static class MRContainerPartitioner extends Partitioner<MRContainer,MRContainer> {
+ final public int getPartition ( MRContainer key, MRContainer value, int numPartitions ) {
+ return Math.abs(key.hashCode()) % numPartitions;
+ }
+ }
+
+ /** The mapper of the MapReduce operation */
+ private final static class MRMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
+ private static Function map_fnc; // the map function
+ private static Function combine_fnc; // the combiner function
+ private static Hashtable<MRData,MRData> hashTable; // in-mapper combiner
+ private static int index;
+ private static Tuple tkey = new Tuple(2);
+ private static Bag tbag = new Bag(2);
+ private static MRContainer ckey = new MRContainer(new MR_int(0));
+ private static MRContainer cvalue = new MRContainer(new MR_int(0));
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ for ( MRData e: (Bag)map_fnc.eval(value.data()) ) {
+ Tuple p = (Tuple)e;
+ if (hashTable == null) {
+ ckey.set(p.first());
+ cvalue.set(p.second());
+ context.write(ckey,cvalue);
+ } else {
+ // in-mapper combiner
+ MRData old = hashTable.get(p.first());
+ if (old == null) {
+ if (index++ == Config.map_cache_size)
+ flush_table(context);
+ hashTable.put(p.first(),p.second());
+ } else {
+ tkey.set(0,p.first());
+ tbag.clear();
+ tbag.add_element(p.second()).add_element(old);
+ tkey.set(1,tbag);
+ for ( MRData x: (Bag)combine_fnc.eval(tkey) )
+ hashTable.put(p.first(),x); // normally, done once
+ }
+ }
+ }
+ }
+
+ private static void flush_table ( Context context ) throws IOException, InterruptedException {
+ Enumeration<MRData> en = hashTable.keys();
+ while (en.hasMoreElements()) {
+ MRData key = en.nextElement();
+ ckey.set(key);
+ MRData value = hashTable.get(key);
+ cvalue.set(value);
+ if (value != null)
+ context.write(ckey,cvalue);
+ };
+ index = 0;
+ hashTable.clear();
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.mapper"));
+ map_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.combiner"));
+ hashTable = null;
+ if (code != null && !code.equals(new VariableLeaf("null"))) {
+ combine_fnc = functional_argument(conf,code);
+ hashTable = new Hashtable<MRData,MRData>(Config.map_cache_size);
+ index = 0;
+ }
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the mapper plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (hashTable != null)
+ flush_table(context);
+ hashTable = null; // garbage-collect it
+ super.cleanup(context);
+ }
+ }
+
+ /** The reducer of the MapReduce operation */
+ private final static class MRReducer extends Reducer<MRContainer,MRContainer,MRContainer,MRContainer> {
+ private static String counter; // a Hadoop user-defined counter used in the repeat operation
+ private static Function reduce_fnc; // the reduce function
+ private static Bag s = new Bag(); // a cached bag of input fragments
+ private static Function acc_fnc; // aggregator
+ private static MRData result; // aggregation result
+ private static boolean streamed = false;
+ private static Tuple pair = new Tuple(2);
+ private static MRContainer container = new MRContainer(new MR_int(0));
+
+ private void write ( MRContainer key, MRData value, Context context )
+ throws IOException, InterruptedException {
+ if (result != null) { // aggregation
+ pair.set(0,result);
+ pair.set(1,value);
+ result = acc_fnc.eval(pair);
+ } else if (counter.equals("-")) {
+ container.set(value);
+ context.write(key,container);
+ } else { // increment the repetition counter if the repeat condition is true
+ Tuple t = (Tuple)value;
+ if (((MR_bool)t.second()).get())
+ context.getCounter("mrql",counter).increment(1);
+ container.set(t.first());
+ context.write(key,container);
+ }
+ }
+
+ @Override
+ public void reduce ( MRContainer key, Iterable<MRContainer> values, Context context )
+ throws IOException, InterruptedException {
+ if (!streamed) { // store the values in a Bag and then reduce
+ s.clear();
+ for ( MRContainer val: values )
+ s.add(val.data());
+ pair.set(0,key.data());
+ pair.set(1,s);
+ for ( MRData e: (Bag)reduce_fnc.eval(pair) )
+ write(key,e,context);
+ } else { // it accesses the values in stream-like fashion
+ final Iterator<MRContainer> iterator = values.iterator();
+ Bag s = new Bag(new BagIterator() {
+ public boolean hasNext () {
+ return iterator.hasNext();
+ }
+ public MRData next () {
+ return iterator.next().data();
+ }
+ });
+ pair.set(0,key.data());
+ pair.set(1,s);
+ for ( MRData e: (Bag)reduce_fnc.eval(pair) )
+ write(key,e,context);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.reducer"));
+ reduce_fnc = functional_argument(conf,code);
+ streamed = PlanGeneration.streamed_MapReduce_reducer(code);
+ if (conf.get("mrql.zero") != null) {
+ code = Tree.parse(conf.get("mrql.zero"));
+ result = Interpreter.evalE(code);
+ code = Tree.parse(conf.get("mrql.accumulator"));
+ acc_fnc = functional_argument(conf,code);
+ } else result = null;
+ counter = conf.get("mrql.counter");
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the reduce plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (result != null) // emit the result of aggregation
+ context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
+ super.cleanup(context);
+ }
+ }
+
+ /**
+ * The MapReduce physical operator
+ * @param map_fnc the mapper function
+ * @param combine_fnc optional in-mapper combiner function
+ * @param reduce_fnc the reducer function
+ * @param acc_fnc optional accumulator function
+ * @param zero optional the zero value for the accumulator
+ * @param source the input data source
+ * @param num_reduces number of reducers
+ * @param stop_counter optional counter used in repeat operation
+ * @param orderp does the result need to be ordered?
+ * @return a new data source that contains the result
+ */
+ public final static DataSet mapReduce ( Tree map_fnc, // mapper function
+ Tree combine_fnc, // optional in-mapper combiner function
+ Tree reduce_fnc, // reducer function
+ Tree acc_fnc, // optional accumulator function
+ Tree zero, // optional the zero value for the accumulator
+ DataSet source, // input data source
+ int num_reduces, // number of reducers
+ String stop_counter, // optional counter used in repeat operation
+ boolean orderp ) // does the result need to be ordered?
+ throws Exception {
+ String newpath = new_path(conf);
+ conf.set("mrql.mapper",map_fnc.toString());
+ if (combine_fnc != null)
+ conf.set("mrql.combiner",combine_fnc.toString());
+ conf.set("mrql.reducer",reduce_fnc.toString());
+ if (zero != null) { // will use in-mapper combiner
+ conf.set("mrql.accumulator",acc_fnc.toString());
+ conf.set("mrql.zero",zero.toString());
+ // the in-mapper combiner likes large data splits
+ conf.set("mapred.min.split.size","268435456"); // 256 MBs
+ } else conf.set("mrql.zero","");
+ conf.set("mrql.counter",stop_counter);
+ Job job = new Job(conf,newpath);
+ distribute_compiled_arguments(job.getConfiguration());
+ job.setJarByClass(MapReducePlan.class);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setPartitionerClass(MRContainerPartitioner.class);
+ job.setSortComparatorClass(MRContainerKeyComparator.class);
+ job.setGroupingComparatorClass(MRContainerKeyComparator.class);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ for (DataSource p: source.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MRMapper.class);
+ FileOutputFormat.setOutputPath(job,new Path(newpath));
+ job.setReducerClass(MRReducer.class);
+ if (Config.trace && PlanGeneration.streamed_MapReduce_reducer(reduce_fnc))
+ System.err.println("*** Streamed MapReduce reducer");
+ if (num_reduces > 0)
+ job.setNumReduceTasks(num_reduces);
+ job.waitForCompletion(true);
+ long c = (stop_counter.equals("-")) ? 0
+ : job.getCounters().findCounter("mrql",stop_counter).getValue();
+ DataSource s = new BinaryDataSource(newpath,conf);
+ s.to_be_merged = orderp;
+ return new DataSet(s,c,outputRecords(job));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/MapReduceParsedInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/MapReduceParsedInputFormat.java b/mapreduce/src/main/java/org/apache/mrql/MapReduceParsedInputFormat.java
new file mode 100644
index 0000000..e588dbd
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/MapReduceParsedInputFormat.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.Trees;
+import java.io.*;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+
+/** A FileInputFormat for text files (CVS, XML, JSON, ...) */
+final public class MapReduceParsedInputFormat extends MapReduceMRQLFileInputFormat {
+
+ public static class ParsedRecordReader extends RecordReader<MRContainer,MRContainer> {
+ final FSDataInputStream fsin;
+ final long start;
+ final long end;
+ Iterator<MRData> result;
+ MRData data;
+ Parser parser;
+
+ public ParsedRecordReader ( FileSplit split,
+ TaskAttemptContext context,
+ Class<? extends Parser> parser_class,
+ Trees args ) throws IOException {
+ Configuration conf = context.getConfiguration();
+ start = split.getStart();
+ end = start + split.getLength();
+ Path file = split.getPath();
+ FileSystem fs = file.getFileSystem(conf);
+ fsin = fs.open(split.getPath());
+ try {
+ parser = parser_class.newInstance();
+ } catch (Exception ex) {
+ throw new Error("Unrecognized parser:"+parser_class);
+ };
+ parser.initialize(args);
+ parser.open(fsin,start,end);
+ result = null;
+ }
+
+ public boolean nextKeyValue () throws IOException {
+ while (result == null || !result.hasNext()) {
+ String s = parser.slice();
+ if (s == null)
+ return false;
+ result = parser.parse(s).iterator();
+ };
+ data = (MRData)result.next();
+ return true;
+ }
+
+ public MRContainer getCurrentKey () throws IOException {
+ return new MRContainer(new MR_long(fsin.getPos()));
+ }
+
+ public MRContainer getCurrentValue () throws IOException {
+ return new MRContainer(data);
+ }
+
+ public long getPos () throws IOException { return fsin.getPos(); }
+
+ public void close () throws IOException { fsin.close(); }
+
+ public float getProgress () throws IOException {
+ if (end == start)
+ return 0.0f;
+ else return Math.min(1.0f, (getPos() - start) / (float)(end - start));
+ }
+
+ public void initialize ( InputSplit split, TaskAttemptContext context ) throws IOException { }
+ }
+
+ public RecordReader<MRContainer,MRContainer>
+ createRecordReader ( InputSplit split,
+ TaskAttemptContext context ) throws IOException, InterruptedException {
+ Configuration conf = context.getConfiguration();
+ String path = ((FileSplit)split).getPath().toString();
+ ParsedDataSource ds = (ParsedDataSource)DataSource.get(path,conf);
+ return new ParsedRecordReader((FileSplit)split,context,ds.parser,(Trees)ds.args);
+ }
+
+ /** Find the parser associated with each file in the path and parse the file,
+ * inserting all results into a Bag. The Bag is lazily constructed.
+ * @param path the path directory with the files
+ * @return a Bag that contains all data
+ */
+ public Bag materialize ( final Path path ) throws IOException {
+ Configuration conf = Plan.conf;
+ ParsedDataSource ds = (ParsedDataSource)DataSource.get(path.toString(),conf);
+ FileSystem fs = path.getFileSystem(conf);
+ FSDataInputStream fsin = fs.open(path);
+ Parser p;
+ try {
+ p = ds.parser.newInstance();
+ } catch (Exception ex) {
+ throw new Error("Unrecognized parser:"+ds.parser);
+ };
+ final Parser parser = p;
+ parser.initialize(ds.args);
+ parser.open(fsin,0,Long.MAX_VALUE);
+ return new Bag(new BagIterator () {
+ Iterator<MRData> iter;
+ public boolean hasNext () {
+ while (iter == null || !iter.hasNext()) {
+ String line = parser.slice();
+ if (line == null)
+ return false;
+ iter = parser.parse(line).iterator();
+ };
+ return true;
+ }
+ public MRData next () {
+ return iter.next();
+ }
+ });
+ }
+}
[16/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Node.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Node.java b/gen/src/main/java/org/apache/mrql/gen/Node.java
new file mode 100644
index 0000000..8bc55d4
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Node.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+
+
+final public class Node extends Tree {
+ public String name;
+ public Trees children;
+
+ public Node ( String name, Trees children ) {
+ super();
+ this.name = Tree.add(name);
+ this.children = children;
+ }
+
+ public Node ( String name ) {
+ super();
+ this.name = name;
+ children = Trees.nil;
+ }
+
+ final public String name () { return name; }
+
+ final public Trees children () { return children; }
+
+ public boolean equals ( Tree e ) {
+ return (e instanceof Node)
+ && name == ((Node) e).name
+ && children.equals(((Node) e).children);
+ }
+
+ protected int size () {
+ return name().length()+children().size();
+ }
+
+ public String toString () {
+ if (Character.isLetter(name.charAt(0))
+ || !(children().length()==2))
+ return name + children().toString();
+ else return "(" + children().head().toString() + name
+ + children().tail().head().toString() + ")";
+ }
+
+ public String pretty ( int position ) {
+ if (Character.isLetter(name.charAt(0))
+ || !(children().length()==2))
+ return name + children().pretty(position+name.length());
+ else return "(" + children().head().toString() + name
+ + children().tail().head().toString() + ")";
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ out.defaultWriteObject();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ name = Tree.add(name);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/StringLeaf.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/StringLeaf.java b/gen/src/main/java/org/apache/mrql/gen/StringLeaf.java
new file mode 100644
index 0000000..5aa41c9
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/StringLeaf.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+
+
+final public class StringLeaf extends Tree {
+ public String value;
+
+ public StringLeaf ( String s ) {
+ super();
+ value = s;
+ }
+
+ public String value () { return value; }
+
+ public boolean equals ( Tree e ) {
+ return (e instanceof StringLeaf)
+ && value.equals(((StringLeaf) e).value);
+ }
+
+ protected int size () { return value.length()+2; }
+
+ public String toString () {
+ return "\"" + value + "\"";
+ }
+
+ public String pretty ( int position ) {
+ return toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/SymbolTable.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/SymbolTable.java b/gen/src/main/java/org/apache/mrql/gen/SymbolTable.java
new file mode 100644
index 0000000..52ea868
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/SymbolTable.java
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+import java.util.Iterator;
+
+
+/* A symbol in the symbol table */
+class SymbolCell {
+ String name;
+ Tree binding;
+ SymbolCell next;
+ SymbolCell ( String n, Tree v, SymbolCell r ) { name=n; binding=v; next=r; }
+}
+
+
+final class SymbolTableIterator implements Iterator<String> {
+ SymbolTable table;
+ SymbolTable original_table;
+
+ SymbolTableIterator ( SymbolTable t ) {
+ table = new SymbolTable();
+ original_table = t;
+ for ( int i = 0; i < t.symbol_table_size; i++ )
+ table.symbol_table[i] = t.symbol_table[i];
+ for ( int i = 0; i < t.scope_stack.length; i++ )
+ table.scope_stack[i] = t.scope_stack[i];
+ table.scope_stack_top = t.scope_stack_top;
+ }
+
+ public boolean hasNext () {
+ if (table.scope_stack_top <= 0)
+ return false;
+ int n = table.scope_stack[table.scope_stack_top-1];
+ if (n < 0 || !table.symbol_table[n].binding.equals(original_table.lookup(table.symbol_table[n].name))) {
+ table.scope_stack_top--;
+ return hasNext();
+ };
+ return true;
+ }
+
+ public String next () {
+ int n = table.scope_stack[--table.scope_stack_top];
+ String var = table.symbol_table[n].name;
+ table.symbol_table[n] = table.symbol_table[n].next;
+ return var;
+ }
+
+ public void remove () {}
+}
+
+
+public class SymbolTable implements Iterable<String> {
+ final static int symbol_table_size = 997;
+ final static int initial_scope_stack_length = 1000;
+
+ SymbolCell[] symbol_table;
+ int[] scope_stack;
+ int scope_stack_top = 0;
+
+ public SymbolTable () {
+ symbol_table = new SymbolCell[symbol_table_size];
+ scope_stack = new int[initial_scope_stack_length];
+ scope_stack_top = 0;
+ for (int i = 0; i < symbol_table_size; i++)
+ symbol_table[i] = null;
+ }
+
+ public Iterator<String> iterator () {
+ return new SymbolTableIterator(this);
+ }
+
+ /* a hashing function for strings */
+ int hash ( String s ) {
+ return Math.abs(s.hashCode()) % symbol_table_size;
+ }
+
+ /* insert a new item in the symbol table */
+ public void insert ( String key, Tree binding ) {
+ int loc = hash(key);
+ symbol_table[loc] = new SymbolCell(key,binding,symbol_table[loc]);
+ if (scope_stack_top >= scope_stack.length) {
+ int[] v = new int[scope_stack.length*2];
+ for ( int i = 0; i < scope_stack.length; i++ )
+ v[i] = scope_stack[i];
+ scope_stack = v;
+ };
+ scope_stack[scope_stack_top++] = loc;
+ }
+
+ /* replace an item with a given name in the symbol table */
+ public void replace ( String key, Tree binding ) {
+ int loc = hash(key);
+ for (SymbolCell s = symbol_table[loc]; s != null; s=s.next)
+ if (s.name.equals(key))
+ s.binding = binding;
+ }
+
+ /* remove an item with a given name from the symbol table */
+ public boolean remove ( String key ) {
+ int loc = hash(key);
+ SymbolCell prev = symbol_table[loc];
+ if (prev == null)
+ return false;
+ if (prev.name.equals(key)) {
+ symbol_table[loc] = prev.next;
+ return true;
+ };
+ for (SymbolCell s = prev.next; s != null; s=s.next, prev=prev.next)
+ if (s.name.equals(key)) {
+ prev.next = s.next;
+ return true;
+ };
+ return false;
+ }
+
+ /* lookup for an item in the symbol table */
+ public Tree lookup ( String key ) {
+ int loc = hash(key);
+ for (SymbolCell s = symbol_table[loc]; s != null; s=s.next)
+ if (s.name.equals(key))
+ return s.binding;
+ return null; // if not found
+ }
+
+ /* return true if key is local */
+ public boolean is_local ( String key ) {
+ int loc = hash(key);
+ int i = 0;
+ for ( SymbolCell s = symbol_table[loc]; s != null; s = s.next, i++ )
+ if (s.name.equals(key)) {
+ int k = 0;
+ for ( int j = scope_stack_top-1; j >= 0 && scope_stack[j] >= 0; j--)
+ if (scope_stack[j] == loc)
+ if (k++ == i)
+ return true;
+ return false;
+ };
+ return false; // if not found
+ }
+
+ /* start a new environment */
+ public void begin_scope () {
+ if (scope_stack_top >= scope_stack.length) {
+ int[] v = new int[scope_stack.length*2];
+ for ( int i = 0; i < scope_stack.length; i++ )
+ v[i] = scope_stack[i];
+ scope_stack = v;
+ };
+ scope_stack[scope_stack_top++] = -1;
+ }
+
+ /* pop the last environment */
+ public void end_scope () {
+ int i = scope_stack_top-1;
+ for (; scope_stack[i]>=0 && i>0; i--) {
+ int loc = scope_stack[i];
+ symbol_table[loc] = symbol_table[loc].next;
+ };
+ scope_stack_top = i;
+ }
+
+ /* display the content of the symbol table */
+ public void display () {
+ SymbolCell[] s = new SymbolCell[symbol_table_size];
+ for (int i = 0; i<symbol_table_size; i++)
+ s[i] = symbol_table[i];
+ for (int i = scope_stack_top-1; i>=0; i--)
+ if (scope_stack[i] == -1)
+ System.out.println("----------------");
+ else {
+ SymbolCell c = s[scope_stack[i]];
+ s[scope_stack[i]] = c.next;
+ System.out.println(c.name + ": " + c.binding);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Tree.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Tree.java b/gen/src/main/java/org/apache/mrql/gen/Tree.java
new file mode 100644
index 0000000..7ddc582
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Tree.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java_cup.runtime.*;
+import java.io.*;
+import java.util.HashMap;
+
+
+abstract public class Tree implements Serializable {
+
+ /* the current line number in the parsed file */
+ public static int line_number = 0;
+
+ /* the current char position in the parsed file */
+ public static int position_number = 0;
+
+ /* true when Trees are parsed rather than processed */
+ public static boolean parsed = false;
+
+ /* the line number of the Tree in the parsed file */
+ public int line;
+
+ /* the char position of the Tree in the parsed file */
+ public int position;
+
+ Tree () { line = line_number; position = position_number; }
+
+ /* deep equality */
+ public abstract boolean equals ( Tree e );
+
+ final public boolean is_node () { return (this instanceof Node); }
+ final public boolean is_variable () { return (this instanceof VariableLeaf); }
+ final public boolean is_long () { return (this instanceof LongLeaf); }
+ final public boolean is_string () { return (this instanceof StringLeaf); }
+ final public boolean is_double () { return (this instanceof DoubleLeaf); }
+
+ final public String variableValue () { return (this instanceof VariableLeaf) ? ((VariableLeaf)this).value() : ""; }
+ final public long longValue () { return (this instanceof LongLeaf) ? ((LongLeaf)this).value() : (long)0; }
+ final public String stringValue () { return (this instanceof StringLeaf) ? ((StringLeaf)this).value() : ""; }
+ final public double doubleValue () { return (this instanceof DoubleLeaf) ? ((DoubleLeaf)this).value() : (double)0.0; }
+
+ /* size used for pretty() */
+ protected abstract int size ();
+
+ /* print the Tree into a string */
+ public abstract String toString ();
+
+ /* pretty-print the Tree padded with position space characters */
+ public abstract String pretty ( int position );
+
+ private static Tree fix_tree ( Tree e ) {
+ if (e instanceof Node) {
+ Trees cs = Trees.nil;
+ for ( Tree a: ((Node) e).children().tail() )
+ cs = cs.append(fix_tree(a));
+ return new Node(((VariableLeaf)(((Node) e).children().head())).value(),cs);
+ } else return e;
+ }
+
+ /* the inverse of toString() */
+ final public static synchronized Tree parse ( String s ) throws Exception {
+ GenParser.scanner = new GenLex(new StringReader("#<"+s+">"));
+ GenParser.out = new PrintStream(new ByteArrayOutputStream());
+ new GenParser(GenParser.scanner).parse();
+ return fix_tree(GenParser.parse_tree);
+ }
+
+ private static HashMap<String,String> names = new HashMap<String,String>(1000);
+
+ public static String add ( String s ) {
+ String ns = names.get(s);
+ if (ns == null) {
+ names.put(s,s);
+ return s;
+ } else return ns;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Trees.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Trees.java b/gen/src/main/java/org/apache/mrql/gen/Trees.java
new file mode 100644
index 0000000..86f65ca
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Trees.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.util.Iterator;
+import java.io.*;
+
+
+final class TreeIterator implements Iterator<Tree> {
+ Trees trees;
+
+ TreeIterator ( Trees trees ) { this.trees = trees; }
+
+ public boolean hasNext () { return trees.tail != null; }
+
+ public Tree next () {
+ Tree res = trees.head;
+ trees = trees.tail;
+ return res;
+ }
+
+ public void remove () { trees = trees.tail; }
+}
+
+
+final public class Trees implements Iterable<Tree>, Serializable {
+ private final static int screen_size = 100;
+ public Tree head;
+ public Trees tail;
+
+ public Trees ( Tree head, Trees tail ) {
+ if (tail == null)
+ throw new Error("Gen: an empty list of nodes must be nil, not null");
+ this.head = head;
+ this.tail = tail;
+ }
+
+ public Trees () {
+ head = null;
+ tail = null;
+ }
+
+ public final static Trees nil = new Trees();
+
+ public Trees ( Tree head ) {
+ this.head = head;
+ tail = nil;
+ }
+
+ public Tree head () {
+ if (tail == null)
+ throw new Error("Gen: tried to retrieve the head of an empty list of nodes");
+ return head;
+ }
+
+ public Trees tail () {
+ if (tail == null)
+ throw new Error("Gen: tried to retrieve the tail of an empty list of nodes");
+ return tail;
+ }
+
+ public boolean is_empty () {
+ return (tail == null);
+ }
+
+ /* number of nodes */
+ public int length () {
+ int n = 0;
+ for (Trees r = this; !r.is_empty(); r = r.tail)
+ n += 1;
+ return n;
+ }
+
+ /* put an Tree e at the beginning of the nodes */
+ public Trees cons ( Tree e ) {
+ return new Trees(e,this);
+ }
+
+ /* put an Tree e at the end of the nodes */
+ public Trees append ( Tree e ) {
+ if (is_empty())
+ return new Trees(e);
+ else {
+ Trees temp = new Trees(e,new Trees(e));
+ Trees res = temp;
+ for (Trees r = this; !r.is_empty(); r = r.tail) {
+ temp.tail = temp.tail.cons(r.head);
+ temp = temp.tail;
+ };
+ return res.tail;
+ }
+ }
+
+ /* append two lists of nodes */
+ public Trees append ( Trees s ) {
+ if (is_empty())
+ return s;
+ else if (s.is_empty())
+ return this;
+ else {
+ Trees temp = s.cons(s.head);
+ Trees res = temp;
+ for (Trees r = this; !r.is_empty(); r = r.tail)
+ { temp.tail = temp.tail.cons(r.head);
+ temp = temp.tail;
+ }
+ return res.tail;
+ }
+ }
+
+ /* reverse the order of nodes */
+ public Trees reverse () {
+ Trees res = nil;
+ for (Trees r = this; !r.is_empty(); r = r.tail)
+ res = res.cons(r.head);
+ return res;
+ }
+
+ /* is e one of the nodes? */
+ public boolean member ( Tree e ) {
+ for (Trees r = this; !r.is_empty(); r = r.tail)
+ if (r.head.equals(e))
+ return true;
+ return false;
+ }
+
+ /* return the nth node */
+ public Tree nth ( int n ) {
+ Trees r = this;
+ for (int i = 0; !r.is_empty() && i < n; r = r.tail(), i++)
+ ;
+ if (r.is_empty())
+ throw new Error("Gen: tried to retrieve a nonexistent nth element from a list of nodes");
+ else return r.head;
+ }
+
+ /* deep equality */
+ public boolean equals ( Trees s ) {
+ Trees n = this;
+ Trees m = s;
+ for(; n.tail != null && m.tail != null; n = n.tail, m = m.tail )
+ if (!n.head.equals(m.head))
+ return false;
+ return (m.tail == null) && (n.tail == null);
+ }
+
+ protected int size () {
+ int n = 1;
+ for (Trees r = this; !r.is_empty(); r = r.tail)
+ n += r.head.size()+1;
+ return n;
+ }
+
+ public Iterator<Tree> iterator () { return new TreeIterator(this); }
+
+ /* print the nodes */
+ public String toString () {
+ if (is_empty())
+ return "()";
+ String s = "(" + head;
+ for (Trees r = tail; !r.is_empty(); r = r.tail)
+ s = s + "," + r.head;
+ return s + ")";
+ }
+
+ /* pretty-print the nodes */
+ public String pretty ( int position ) {
+ if (is_empty() || (position+size() <= screen_size))
+ return toString();
+ String s = "(" + head.pretty(position+1);
+ for (Trees r=tail; !r.is_empty(); r=r.tail) {
+ s = s + ",\n";
+ for (int i=0; i<position+1; i++)
+ s = s + " ";
+ s = s + r.head.pretty(position+1);
+ };
+ return s + ")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/VariableLeaf.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/VariableLeaf.java b/gen/src/main/java/org/apache/mrql/gen/VariableLeaf.java
new file mode 100644
index 0000000..c4c0ed9
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/VariableLeaf.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+
+
+final public class VariableLeaf extends Tree {
+ public String value;
+
+ public VariableLeaf ( String s ) {
+ super();
+ value = Tree.add(s);
+ }
+
+ final public String value () { return value; }
+
+ public boolean equals ( Tree e ) {
+ return (e instanceof VariableLeaf)
+ && value == ((VariableLeaf) e).value;
+ }
+
+ protected int size () { return value.length(); }
+
+ public String toString () { return value; }
+
+ public String pretty ( int position ) { return value; }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ out.defaultWriteObject();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ value = Tree.add(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/gen.cup
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/gen.cup b/gen/src/main/java/org/apache/mrql/gen/gen.cup
new file mode 100644
index 0000000..91432ba
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/gen.cup
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java_cup.runtime.*;
+import java.util.Stack;
+
+parser code {:
+
+ static int[] tokens = {
+ GenSym.error, GenSym.CHAR, GenSym.ID, GenSym.CSTRING, GenSym.LONG, GenSym.DOUBLE,
+ GenSym.META, GenSym.MATCH, GenSym.CASE, GenSym.IS, GenSym.COLON, GenSym.COMMA,
+ GenSym.GT, GenSym.LP, GenSym.RP, GenSym.LB, GenSym.RB, GenSym.LSB, GenSym.RSB,
+ GenSym.BQ, GenSym.BQP, GenSym.DOTSP, GenSym.ANY, GenSym.OPER, GenSym.FAIL
+ };
+
+ static String[] token_names = {
+ "error", "character", "identifier", "string", "integer", "float",
+ "#<", "match", "case", "is", ":", ",", ">", "(", ")", "[", "]", "{", "}",
+ "`", "`(", "...(", "_", "operator", "fail"
+ };
+
+ public static String print ( Symbol s ) {
+ for (int i=0; i<tokens.length; i++)
+ if (tokens[i]==s.sym)
+ { String res = token_names[i] + " ";
+ if (s.value==null)
+ return res;
+ else if (s.value instanceof Long)
+ return res + ((Long) s.value).longValue();
+ else if (s.value instanceof Double)
+ return res + ((Double) s.value).doubleValue();
+ else if (s.value instanceof String)
+ return res + (String) s.value;
+ }
+ return "?";
+ }
+
+ public static java.io.PrintStream out;
+
+ public static GenLex scanner;
+
+ public static Tree parse_tree;
+
+ public static int newlines = 0;
+
+ public static int lcase = 0;
+
+ public static Stack<String> labels = new Stack<String>();
+
+ public void syntax_error ( Symbol token ) {
+ throw new Error("*** Syntax Error: " + print(token) + " (line: " + scanner.line_pos()
+ + ", position: " + scanner.char_pos() + ")");
+ }
+
+:};
+
+/* Terminals (tokens returned by the scanner). */
+terminal META, METAL, MATCH, CASE, FAIL, IS, DOTS, COLON, COMMA, GT, LP, RP, LB, RB, LSB, RSB, BQ, BQP, DOTSP, ANY;
+
+terminal String CHAR;
+terminal String ID;
+terminal Long LONG;
+terminal Double DOUBLE;
+terminal String CSTRING;
+terminal String OPER;
+
+non terminal Tree exp, name, variable, rest;
+non terminal Trees expl;
+non terminal String case, cases, schar, jcode, scode;
+non terminal pcode, prog;
+
+precedence left OPER;
+precedence nonassoc META;
+precedence nonassoc MATCH;
+precedence nonassoc CASE;
+precedence nonassoc CSTRING;
+precedence nonassoc CHAR;
+
+start with prog;
+
+prog ::= pcode
+ ;
+pcode ::= scode:s {: parser.out.print(s); :}
+ | pcode scode:s {: parser.out.print(s); :}
+ ;
+jcode ::= scode:s {: RESULT = s; :}
+ | jcode:c scode:s {: RESULT = c+s; :}
+ ;
+scode ::= schar:s {: RESULT = s; :}
+ | LSB RSB {: RESULT = "{}"; :}
+ | LSB jcode:c RSB {: RESULT = "{"+c+"}"; :}
+ | METAL RB {: RESULT = "Trees.nil"; :}
+ | META exp:e GT {: GenParser.parse_tree = e;
+ RESULT = Meta.reify(e);
+ for (int i=0; i < parser.newlines; i++)
+ RESULT += "\n";
+ parser.newlines = 0; :}
+ | METAL expl:r RB {: RESULT = Meta.reify(new Node("Node",
+ new Trees(new VariableLeaf("F"),r)));
+ RESULT = RESULT.substring(13,RESULT.length()-1);
+ for (int i=0; i < parser.newlines; i++)
+ RESULT += "\n";
+ parser.newlines = 0; :}
+ | MATCH {: Meta.level++;
+ GenLex.context.new_context();
+ GenLex.context.in_match_body();
+ :}
+ jcode:c LSB cases:el RSB {: RESULT = "{ boolean FOUND_" + Meta.level + " = false; Tree E_"
+ + Meta.level + " = " + c + "; " + el + " }";
+ GenLex.context.close_context(GenParser.scanner);
+ for (int i=0; i < parser.newlines; i++)
+ RESULT += "\n";
+ parser.newlines = 0;
+ Meta.level--; :}
+ | FAIL {: RESULT = "{ FOUND_" + Meta.level + "=false; break " + GenParser.labels.peek() + "; }"; :}
+ ;
+schar ::= CHAR:c {: RESULT = c; :}
+ | CSTRING:s {: RESULT = Crypt.encrypt(s); :}
+ ;
+cases ::= jcode:j case:c {: RESULT = j+c; :}
+ | cases:cl case:c {: RESULT = cl + " if (!FOUND_" + Meta.level + ") " + c; :}
+ ;
+case ::= CASE {: GenParser.labels.push("LCASE_"+GenParser.lcase++); :}
+ exp:e COLON jcode:j {: Condition m = Meta.pattern(e,"E_" + Meta.level);
+ RESULT = "{ " + GenParser.labels.pop() + ": ";
+ RESULT += (m.pred.equals("true")) ? "" : "if (" + m.pred + ") ";
+ for (int i=0; i < parser.newlines; i++)
+ RESULT += "\n";
+ parser.newlines = 0;
+ RESULT += "{ " + m.stmt + " FOUND_" + Meta.level
+ + "=true; " + j + " }";
+ for (int i = 0; i <= m.unmatched_brackets; i++)
+ RESULT += "}";
+ RESULT += ";"; :}
+ ;
+name ::= ID:s {: RESULT = new VariableLeaf(s); :}
+ ;
+variable ::= name:s {: RESULT = s; :}
+ | BQ name:s {: RESULT = new Node("Escape",new Trees(s)); :}
+ | BQP jcode:c RP {: RESULT = new Node("Code",new Trees(new StringLeaf(c))); :}
+ | ANY {: RESULT = new VariableLeaf("_any_"); :}
+ ;
+exp ::= LONG:n {: RESULT = new LongLeaf(n.longValue()); :}
+ | DOUBLE:n {: RESULT = new DoubleLeaf(n.doubleValue()); :}
+ | CSTRING:s {: RESULT = new StringLeaf(s); :}
+ | variable:e {: RESULT = e; :}
+ | variable:e LP RP {: RESULT = new Node("Node",new Trees(e)); :}
+ | variable:e LP expl:el RP {: RESULT = new Node("Node",new Trees(e,el)); :}
+ | BQ name:s LB exp:e RB {: RESULT = new Node("Higher",new Trees(s,new Trees(e))); :}
+ | name:s IS exp:e {: RESULT = new Node("IS",new Trees(s,new Trees(e))); :}
+ | LP exp:e RP {: RESULT = e; :}
+ | exp:e1 OPER:o exp:e2 {: RESULT = new Node("Node",new Trees(new VariableLeaf(o),
+ new Trees(e1,new Trees(e2)))); :}
+ ;
+expl ::= exp:e {: RESULT = new Trees(e); :}
+ | rest:e {: RESULT = new Trees(e); :}
+ | expl:el COMMA rest:e {: RESULT = el.append(e); :}
+ | expl:el COMMA exp:e {: RESULT = el.append(e); :}
+ ;
+rest ::= DOTS name:n {: RESULT = new Node("Dots",new Trees(n)); :}
+ | DOTSP jcode:c RP {: RESULT = new Node("Dots",new Trees(new StringLeaf(c))); :}
+ | DOTS {: RESULT = new Node("Dots"); :}
+ ;
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/gen.lex
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/gen.lex b/gen/src/main/java/org/apache/mrql/gen/gen.lex
new file mode 100644
index 0000000..06b073d
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/gen.lex
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java_cup.runtime.Symbol;
+
+final class Context {
+ public int[] parentheses;
+ public boolean[] match_begin;
+ public int current;
+ public Context () {
+ current = 0;
+ parentheses = new int[1000];
+ match_begin = new boolean[1000];
+ parentheses[0] = 0;
+ match_begin[0] = false;
+ }
+ public boolean no_parentheses () {
+ return parentheses[current] == 0;
+ }
+ public void new_context () {
+ parentheses[++current] = 0;
+ match_begin[current] = false;
+ }
+ public void close_context ( GenLex lex ) {
+ if (parentheses[current--] > 0)
+ lex.error("Unbalanced Parentheses in Gen construction/pattern");
+ if (current < 0)
+ lex.error("Wrong match statement");
+ }
+ public void open_parenthesis () {
+ parentheses[current]++;
+ }
+ public boolean close_parenthesis () {
+ return (--parentheses[current] == 0) && current > 0 && !match_begin[current];
+ }
+ public void in_match_body () {
+ match_begin[current] = true;
+ }
+ public boolean match_body () { return match_begin[current]; }
+}
+%%
+%state gen
+%class GenLex
+%public
+%line
+%char
+%cup
+
+%eofval{
+ return new java_cup.runtime.Symbol(GenSym.EOF);
+%eofval}
+
+%{
+ public void error ( String msg ) {
+ throw new Error("*** Lexical Error: " + msg + " (line: " + line_pos() + ", position: " + char_pos() + ")");
+ }
+
+ public static Context context = new Context();
+
+ static int prev_char_pos = -1;
+
+ public int line_pos () { return yyline+1; }
+
+ public int char_pos () { return yychar-prev_char_pos; }
+
+ public Symbol symbol ( int s ) {
+ Tree.line_number = line_pos();
+ Tree.position_number = char_pos();
+ //System.out.println(context.parentheses[context.current]+" "+context.match_begin[context.current]+" "+GenParser.print(new Symbol(s)));
+ return new Symbol(s);
+ }
+
+ public Symbol symbol ( int s, Object o ) {
+ Tree.line_number = line_pos();
+ Tree.position_number = char_pos();
+ //System.out.println(context.parentheses[context.current]+" "+context.match_begin[context.current]+" "+GenParser.print(new Symbol(s,o)));
+ return new Symbol(s,o);
+ }
+%}
+
+DIGIT = [0-9]
+ID = [a-zA-Z_][a-zA-Z0-9_]*
+OPER = [!@#$%\^\&*-+=|\\~]+
+NEWLINE = [\n\r]
+DIGITS = {DIGIT}+
+INT = ({DIGIT}|[1-9]{DIGITS}|-{DIGIT}|-[1-9]{DIGITS})
+FRAC = [.]{DIGITS}
+EXP = [eE][+-]?{DIGITS}
+DOUBLE = ({INT}{FRAC}|{INT}{EXP}|{INT}{FRAC}{EXP})
+
+%%
+<gen> {INT} { return symbol(GenSym.LONG,new Long(yytext())); }
+<gen> {DOUBLE} { return symbol(GenSym.DOUBLE,new Double(yytext())); }
+<gen> ":" { yybegin(YYINITIAL);
+ context.close_context(this);
+ return symbol(GenSym.COLON); }
+<gen> "_" { return symbol(GenSym.ANY); }
+<gen> "," { return symbol(GenSym.COMMA); }
+<gen> "`(" { context.new_context();
+ context.open_parenthesis();
+ yybegin(YYINITIAL);
+ return symbol(GenSym.BQP);
+ }
+<gen> "...(" { context.new_context();
+ context.open_parenthesis();
+ yybegin(YYINITIAL);
+ return symbol(GenSym.DOTSP);
+ }
+<gen> "`" { return symbol(GenSym.BQ); }
+<gen> "..." { return symbol(GenSym.DOTS); }
+<gen> "(" { context.open_parenthesis();
+ return symbol(GenSym.LP);
+ }
+<gen> ")" { context.close_parenthesis();
+ return symbol(GenSym.RP);
+ }
+<gen> "[" { context.open_parenthesis();
+ return symbol(GenSym.LB);
+ }
+<gen> "]" { context.close_parenthesis();
+ if (context.no_parentheses())
+ { yybegin(YYINITIAL);
+ context.close_context(this);
+ };
+ return symbol(GenSym.RB);
+ }
+<gen> ">" { yybegin(YYINITIAL);
+ context.close_parenthesis();
+ context.close_context(this);
+ return symbol(GenSym.GT);
+ }
+<gen> "is" { return symbol(GenSym.IS); }
+<gen> {ID} { return symbol(GenSym.ID,yytext()); }
+<gen> {OPER} { return symbol(GenSym.OPER,yytext()); }
+<gen> \/\*[^*/]*\*\/ { for (char c: yytext().toCharArray())
+ if (c=='\n' || c=='\r')
+ GenParser.newlines++;
+ prev_char_pos = yychar; }
+<gen> "//"[^\n\r]* { prev_char_pos = 0; }
+<gen> [ \t\f] {}
+<gen> {NEWLINE} { GenParser.newlines++; prev_char_pos = yychar; }
+<gen> . { error("Illegal character in Gen construction/pattern: "+yytext()); }
+<YYINITIAL> "match" { return symbol(GenSym.MATCH); }
+<YYINITIAL> "case" { if (!context.match_body())
+ return symbol(GenSym.CHAR,yytext());
+ context.new_context();
+ yybegin(gen);
+ return symbol(GenSym.CASE);
+ }
+<YYINITIAL> "fail" { return symbol(GenSym.FAIL); }
+<YYINITIAL> "`" { error("Backquote outside a Gen construction/pattern"); }
+<YYINITIAL> "#<" { context.new_context();
+ context.open_parenthesis();
+ yybegin(gen);
+ return symbol(GenSym.META);
+ }
+<YYINITIAL> "#[" { context.new_context();
+ context.open_parenthesis();
+ yybegin(gen);
+ return symbol(GenSym.METAL);
+ }
+<YYINITIAL> "{" { context.open_parenthesis();
+ if (context.match_body())
+ return symbol(GenSym.LSB);
+ else return symbol(GenSym.CHAR,yytext());
+ }
+<YYINITIAL> "("|"[" { context.open_parenthesis();
+ return symbol(GenSym.CHAR,yytext());
+ }
+<YYINITIAL> "}" { context.close_parenthesis();
+ if (context.match_body())
+ return symbol(GenSym.RSB);
+ else return symbol(GenSym.CHAR,yytext());
+ }
+<YYINITIAL> ")" { if (context.close_parenthesis())
+ { context.close_context(this);
+ yybegin(gen);
+ return symbol(GenSym.RP);
+ } else return symbol(GenSym.CHAR,yytext());
+ }
+<YYINITIAL> "]" { if (context.close_parenthesis())
+ { context.close_context(this);
+ yybegin(gen);
+ return symbol(GenSym.RB);
+ } else return symbol(GenSym.CHAR,yytext());
+ }
+\"[^\"]*\" { return symbol(GenSym.CSTRING,yytext().substring(1,yytext().length()-1)); }
+<YYINITIAL> {ID} { return symbol(GenSym.CHAR,yytext()); }
+<YYINITIAL> {OPER} { return symbol(GenSym.CHAR,yytext()); }
+<YYINITIAL> \/\*[^*/]*\*\/ { prev_char_pos = yychar; return symbol(GenSym.CHAR,yytext()); }
+<YYINITIAL> "//"[^\n\r]* { prev_char_pos = 0; return symbol(GenSym.CHAR,yytext()); }
+<YYINITIAL> [ \t\f] { return symbol(GenSym.CHAR,yytext()); }
+<YYINITIAL> {NEWLINE} { prev_char_pos = yychar; return symbol(GenSym.CHAR,yytext()); }
+<YYINITIAL> . { return symbol(GenSym.CHAR,yytext()); }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/mapreduce/pom.xml b/mapreduce/pom.xml
new file mode 100644
index 0000000..aa8834e
--- /dev/null
+++ b/mapreduce/pom.xml
@@ -0,0 +1,191 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-mr</artifactId>
+ <packaging>jar</packaging>
+ <name>Apache MRQL MapReduce mode</name>
+ <description>Apache MRQL evaluation in MapReduce mode on Apache Hadoop</description>
+ <url>http://mrql.incubator.apache.org/</url>
+ <inceptionYear>2013</inceptionYear>
+
+ <parent>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-parent</artifactId>
+ <version>0.9.1-incubating-SNAPSHOT</version>
+ </parent>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-gen</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <!-- needed for testing in local mode -->
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>2.4</version>
+ </dependency>
+ </dependencies>
+
+ <profiles>
+ <profile>
+ <id>MultipleInputs</id>
+ <activation>
+ <property>
+ <name>MultipleInputs</name>
+ </property>
+ </activation>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.8</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals><goal>add-source</goal></goals>
+ <configuration>
+ <sources>
+ <source>../src/main/java/multipleinputs</source>
+ <source>${project.build.directory}/generated-sources</source>
+ </sources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ </profiles>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.8</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals><goal>add-source</goal></goals>
+ <configuration>
+ <sources>
+ <source>${project.build.directory}/generated-sources</source>
+ </sources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>2.1</version>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <artifactSet>
+ <includes>
+ <include>org.apache.mrql:*</include>
+ </includes>
+ <excludes>
+ <exclude>org.apache.mrql:mrql-gen</exclude>
+ <exclude>org.apache.mrql:mrql-core</exclude>
+ </excludes>
+ </artifactSet>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.4</version>
+ <configuration>
+ <finalName>mrql-mr-${project.version}</finalName>
+ <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
+ <useDefaultManifestFile>true</useDefaultManifestFile>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <version>1.7</version>
+ <executions>
+ <execution>
+ <id>gen</id>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target>
+ <mkdir dir="${project.build.directory}/generated-sources/org/apache/mrql" />
+ <property name="compile_classpath" refid="maven.compile.classpath" />
+ <fileset id="mr.gen.path" dir="src/main/java/org/apache/mrql" includes="*.gen" />
+ <pathconvert pathsep=" " property="mr.gen.files" refid="mr.gen.path" />
+ <java classname="org.apache.mrql.gen.Main" classpath="../lib/mrql-gen-${project.version}.jar:${compile_classpath}">
+ <arg line="${mr.gen.files}" />
+ <arg line="-o" />
+ <arg file="${project.build.directory}/generated-sources/org/apache/mrql" />
+ </java>
+ </target>
+ </configuration>
+ </execution>
+ <execution>
+ <id>validate</id>
+ <phase>test</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target name="validate_hadoop" if="tests" description="Validate all test queries on Apache Hadoop">
+ <property name="runtime_classpath" refid="maven.runtime.classpath" />
+ <echo message="Evaluating test queries in memory (Map-Reduce mode):" />
+ <java classname="org.apache.mrql.Test" classpath="${runtime_classpath}" dir=".." fork="yes">
+ <arg file="../tests/queries" />
+ <arg file="../tests/results/mr-memory" />
+ <arg file="../tests/error_log.txt" />
+ </java>
+ <echo message="Evaluating test queries in Apache Hadoop local mode:" />
+ <java classname="org.apache.mrql.Test" classpath="../lib/mrql-mr-${project.version}.jar:${runtime_classpath}" dir=".." fork="yes" error="/dev/null">
+ <arg line="-local" />
+ <arg file="../tests/queries" />
+ <arg file="../tests/results/hadoop" />
+ <arg file="../tests/error_log.txt" />
+ </java>
+ </target>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/CrossProductOperation.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/CrossProductOperation.java b/mapreduce/src/main/java/org/apache/mrql/CrossProductOperation.java
new file mode 100644
index 0000000..a9216cc
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/CrossProductOperation.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.net.URI;
+import java.util.List;
+import java.util.Vector;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.filecache.DistributedCache;
+
+
+/** The CrossProduct physical operation (similar to block-nested loop) */
+final public class CrossProductOperation extends MapReducePlan {
+
+ /** The mapper for the CrossProduct operation */
+ private final static class crossProductMapper extends Mapper<MRContainer,MRContainer,MRContainer,MRContainer> {
+ private static String counter; // a Hadoop user-defined counter used in the repeat operation
+ private static Function reduce_fnc; // the reduce function
+ private static Function map_fnc; // the mapper function
+ private static DataSet cached_dataset;
+ private final static List<MRData> outer
+ = new Vector<MRData>(Config.map_cache_size); // fix-size cache for the outer
+ private static int index;
+ private static MRContainer last_key;
+ private static URI[] uris;
+ private static Path[] local_paths;
+ private static Function acc_fnc; // aggregator
+ private static MRData result; // aggregation result
+ private static Tuple pair = new Tuple(2);
+ private static MRContainer container = new MRContainer(new MR_int(0));
+
+ private void write ( MRContainer key, MRData value, Context context )
+ throws IOException, InterruptedException {
+ if (result != null) { // aggregation
+ pair.set(0,result);
+ pair.set(1,value);
+ result = acc_fnc.eval(pair);
+ } else if (counter.equals("-")) {
+ container.set(value);
+ context.write(key,container);
+ } else { // increment the repetition counter if the repeat condition is true
+ Tuple t = (Tuple)value;
+ if (((MR_bool)t.second()).get())
+ context.getCounter("mrql",counter).increment(1);
+ container.set(t.first());
+ context.write(key,container);
+ }
+ }
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ try {
+ last_key = key;
+ for ( MRData x: (Bag)map_fnc.eval(value.data()) )
+ if (index++ == Config.map_cache_size) {
+ for ( MRData y: cached_data(context.getConfiguration()) ) {
+ pair.set(1,y);
+ for ( MRData z: outer ) {
+ pair.set(0,z);
+ for ( MRData v: (Bag)reduce_fnc.eval(pair) )
+ write(key,v,context);
+ }
+ };
+ index = 0;
+ outer.clear();
+ } else outer.add(x);
+ } catch (Exception e) {
+ throw new Error("Cannot perform the crossProduct: "+e);
+ }
+ }
+
+ protected Bag cached_data ( final Configuration conf ) {
+ try {
+ Bag res = new Bag();
+ final FileSystem fs = FileSystem.getLocal(conf);
+ for ( int i = 0; i < local_paths.length; i++ ) {
+ // hadoop 0.20.2 distributed cache doesn't work in stand-alone
+ final Path path = (conf.get("mapred.job.tracker").equals("local"))
+ ? new Path(uris[i].toString())
+ : local_paths[i];
+ if (path.getName().endsWith(".jar"))
+ continue;
+ res = res.union(new Bag(new BagIterator () {
+ final SequenceFile.Reader reader = new SequenceFile.Reader(fs,path,conf);
+ final MRContainer key = new MRContainer(new MR_int(0));
+ final MRContainer value = new MRContainer(new MR_int(0));
+ public boolean hasNext () {
+ try {
+ boolean done = reader.next(key,value);
+ if (!done)
+ reader.close();
+ return done;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from distributed cache");
+ }
+ }
+ public MRData next () {
+ return value.data();
+ }
+ }));
+ };
+ return res;
+ } catch (Exception e) {
+ throw new Error("Cannot setup the cross product: "+e);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.reducer"));
+ reduce_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.mapper"));
+ map_fnc = functional_argument(conf,code);
+ if (conf.get("mrql.zero") != null) {
+ code = Tree.parse(conf.get("mrql.zero"));
+ result = Interpreter.evalE(code);
+ code = Tree.parse(conf.get("mrql.accumulator"));
+ acc_fnc = functional_argument(conf,code);
+ } else result = null;
+ counter = conf.get("mrql.counter");
+ uris = DistributedCache.getCacheFiles(conf);
+ local_paths = DistributedCache.getLocalCacheFiles(conf);
+ index = 0;
+ } catch (Exception e) {
+ throw new Error("Cannot setup the crossProduct: "+e);
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (index > 0)
+ try {
+ for ( MRData y: cached_data(context.getConfiguration()) ) {
+ pair.set(1,y);
+ for ( MRData z: outer ) {
+ pair.set(0,z);
+ for ( MRData v: (Bag)reduce_fnc.eval(pair) )
+ write(last_key,v,context);
+ }
+ };
+ } catch (Exception e) {
+ throw new Error("Cannot cleanup the crossProduct: "+e);
+ };
+ index = 0;
+ outer.clear();
+ if (result != null) // emit the result of aggregation
+ context.write(new MRContainer(new MR_int(0)),new MRContainer(result));
+ super.cleanup(context);
+ }
+ }
+
+ /** The CrossProduct physical operator (similar to block-nested loop)
+ * @param mx left mapper
+ * @param my right mapper
+ * @param reduce_fnc reducer
+ * @param acc_fnc optional accumulator function
+ * @param zero optional the zero value for the accumulator
+ * @param X the left source
+ * @param Y the right source (stored in distributed cache)
+ * @param stop_counter optional counter used in repeat operation
+ * @return a new data source that contains the result
+ */
+ public final static DataSet crossProduct ( Tree mx, // left mapper
+ Tree my, // right mapper
+ Tree reduce_fnc, // reducer
+ Tree acc_fnc, // optional accumulator function
+ Tree zero, // optional the zero value for the accumulator
+ DataSet X, // the left source
+ DataSet Y, // the right source (stored in distributed cache)
+ String stop_counter ) // optional counter used in repeat operation
+ throws Exception {
+ DataSet ds = MapOperation.cMap(my,null,null,Y,"-");
+ String newpath = new_path(conf);
+ conf.set("mrql.reducer",reduce_fnc.toString());
+ conf.set("mrql.mapper",mx.toString());
+ if (zero != null) {
+ conf.set("mrql.accumulator",acc_fnc.toString());
+ conf.set("mrql.zero",zero.toString());
+ conf.set("mapred.min.split.size","268435456");
+ } else conf.set("mrql.zero","");
+ conf.set("mrql.counter",stop_counter);
+ Job job = new Job(conf,newpath);
+ distribute_compiled_arguments(job.getConfiguration());
+ job.setJarByClass(MapReducePlan.class);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ PathFilter pf = new PathFilter () { public boolean accept ( Path path ) {
+ return !path.getName().startsWith("_");
+ } };
+ for (DataSource p: ds.source) {
+ Path path = new Path(p.path);
+ for ( FileStatus s: path.getFileSystem(conf).listStatus(path,pf) )
+ DistributedCache.addCacheFile(s.getPath().toUri(),job.getConfiguration());
+ };
+ for (DataSource p: X.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,crossProductMapper.class);
+ FileOutputFormat.setOutputPath(job,new Path(newpath));
+ job.setNumReduceTasks(0);
+ job.waitForCompletion(true);
+ long c = (stop_counter.equals("-")) ? 0
+ : job.getCounters().findCounter("mrql",stop_counter).getValue();
+ return new DataSet(new BinaryDataSource(newpath,conf),c,outputRecords(job));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/mapreduce/src/main/java/org/apache/mrql/GroupByJoinPlan.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/mrql/GroupByJoinPlan.java b/mapreduce/src/main/java/org/apache/mrql/GroupByJoinPlan.java
new file mode 100644
index 0000000..4f93e1e
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/mrql/GroupByJoinPlan.java
@@ -0,0 +1,464 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+
+
+/**
+ * A map-reduce job that captures a join with group-by. Similar to matrix multiplication.<br/>
+ * It captures queries of the form:
+ * <pre>
+ * select r(kx,ky,c(z))
+ * from x in X, y in Y, z = mp(x,y)
+ * where jx(x) = jy(y)
+ * group by (kx,ky): (gx(x),gy(y));
+ * </pre>
+ * where: mp: map function, r: reduce function, c: combine function,
+ * jx: left join key function, jy: right join key function,
+ * gx: left group-by function, gy: right group-by function.
+ * <br/>
+ * Example: matrix multiplication:
+ * <pre>
+ * select ( sum(z), i, j )
+ * from (x,i,k) in X, (y,k,j) in Y, z = x*y
+ * group by (i,j);
+ * </pre>
+ * It uses m*n partitions, so that n/m=|X|/|Y| and a hash table of size |X|/n*|Y|/m can fit in memory M.
+ * That is, n = |X|/sqrt(M), m = |Y|/sqrt(M).
+ * Each partition generates |X|/n*|Y|/m data. It replicates X n times and Y m times.
+ * Uses a hash-table H of size |X|/n*|Y|/m
+ * MapReduce pseudo-code:
+ * <pre>
+ * mapX ( x )
+ * for i = 0,n-1
+ * emit ( ((hash(gx(x)) % m)+m*i, jx(x), 1), (1,x) )
+ *
+ * mapY ( y )
+ * for i = 0,m-1
+ * emit ( ((hash(gy(y)) % n)*m+i, jy(y), 2), (2,y) )
+ * </pre>
+ * mapper output key: (partition,joinkey,tag), value: (tag,data) <br/>
+ * Partitioner: over partition <br/>
+ * GroupingComparator: over partition and joinkey <br/>
+ * SortComparator: major partition, minor joinkey, sub-minor tag <br/>
+ * <pre>
+ * reduce ( (p,_,_), s )
+ * if p != current_partition
+ * flush()
+ * current_partition = p
+ * read x from s first and store it to xs
+ * for each y from the rest of s
+ * for each x in xs
+ * H[(gx(x),gy(y))] = c( H[(gx(x),gy(y))], mp((x,y)) )
+ * </pre>
+ * where flush() is: for each ((kx,ky),v) in H: emit r((kx,ky),v)
+ */
+final public class GroupByJoinPlan extends Plan {
+
+ /** mapper output key: (partition,joinkey,tag) */
+ private final static class GroupByJoinKey implements Writable {
+ public int partition; // one of n*m
+ public byte tag; // 1 or 2
+ public MRData key;
+
+ GroupByJoinKey () {}
+ GroupByJoinKey ( int p, byte t, MRData k ) {
+ partition = p;
+ tag = t;
+ key = k;
+ }
+
+ public void write ( DataOutput out ) throws IOException {
+ out.writeByte(tag);
+ WritableUtils.writeVInt(out,partition);
+ key.write(out);
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ tag = in.readByte();
+ partition = WritableUtils.readVInt(in);
+ key = MRContainer.read(in);
+ }
+
+ public String toString () { return "["+partition+","+tag+","+key+"]"; }
+ }
+
+ /** partition based on key.partition only */
+ private final static class GroupByJoinPartitioner extends Partitioner<GroupByJoinKey,MRContainer> {
+ final public int getPartition ( GroupByJoinKey key, MRContainer value, int numPartitions ) {
+ return key.partition % numPartitions;
+ }
+ }
+
+ /** sorting with major order key.partition, minor key.key, minor key.tag */
+ private final static class GroupByJoinSortComparator implements RawComparator<GroupByJoinKey> {
+ int[] container_size;
+
+ public GroupByJoinSortComparator () {
+ container_size = new int[1];
+ }
+
+ final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
+ try {
+ int c = WritableComparator.readVInt(x,xs+1)-WritableComparator.readVInt(y,ys+1);
+ if (c != 0)
+ return c;
+ int tsize = 1+WritableUtils.decodeVIntSize(x[xs+1]);
+ c = MRContainer.compare(x,xs+tsize,xl-tsize,y,ys+tsize,yl-tsize,container_size);
+ if (c != 0)
+ return c;
+ return x[xs] - y[ys];
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ final public int compare ( GroupByJoinKey x, GroupByJoinKey y ) {
+ int c = x.partition - y.partition;
+ if (c != 0)
+ return c;
+ c = x.key.compareTo(y.key);
+ if (c != 0)
+ return c;
+ return x.tag - y.tag;
+ }
+ }
+
+ /** grouping by key.partition and key.key */
+ private final static class GroupByJoinGroupingComparator implements RawComparator<GroupByJoinKey> {
+ int[] container_size;
+
+ public GroupByJoinGroupingComparator() {
+ container_size = new int[1];
+ }
+
+ final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
+ try {
+ int c = WritableComparator.readVInt(x,xs+1)-WritableComparator.readVInt(y,ys+1);
+ if (c != 0)
+ return c;
+ int tsize = 1+WritableUtils.decodeVIntSize(x[xs+1]);
+ return MRContainer.compare(x,xs+tsize,xl-tsize,y,ys+tsize,yl-tsize,container_size);
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ final public int compare ( GroupByJoinKey x, GroupByJoinKey y ) {
+ int c = x.partition - y.partition;
+ return (c != 0) ? c : x.key.compareTo(y.key);
+ }
+ }
+
+ /** the left GroupByJoin mapper */
+ private final static class MapperLeft extends Mapper<MRContainer,MRContainer,GroupByJoinKey,MRContainer> {
+ private static int n, m;
+ private static Function left_join_key_fnc;
+ private static Function left_groupby_fnc;
+ private static GroupByJoinKey ckey = new GroupByJoinKey(0,(byte)1,new MR_int(0));
+ private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(1));
+ private static MRContainer cvalue = new MRContainer(tvalue);
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ MRData data = value.data();
+ for ( int i = 0; i < n; i++ ) {
+ ckey.partition = (left_groupby_fnc.eval(data).hashCode() % m)+m*i;
+ ckey.key = left_join_key_fnc.eval(data);
+ tvalue.set(1,data);
+ context.write(ckey,cvalue);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.join.key.left"));
+ left_join_key_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.groupby.left"));
+ left_groupby_fnc = functional_argument(conf,code);
+ m = conf.getInt("mrql.m",1);
+ n = conf.getInt("mrql.n",1);
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the left mapper plan");
+ }
+ }
+ }
+
+ /** the right GroupByJoin mapper */
+ private final static class MapperRight extends Mapper<MRContainer,MRContainer,GroupByJoinKey,MRContainer> {
+ private static int n, m;
+ private static Function right_join_key_fnc;
+ private static Function right_groupby_fnc;
+ private static GroupByJoinKey ckey = new GroupByJoinKey(0,(byte)2,new MR_int(0));
+ private static Tuple tvalue = (new Tuple(2)).set(0,new MR_byte(2));
+ private static MRContainer cvalue = new MRContainer(tvalue);
+
+ @Override
+ public void map ( MRContainer key, MRContainer value, Context context )
+ throws IOException, InterruptedException {
+ MRData data = value.data();
+ for ( int i = 0; i < m; i++ ) {
+ ckey.partition = (right_groupby_fnc.eval(data).hashCode() % n)*m+i;
+ ckey.key = right_join_key_fnc.eval(data);
+ tvalue.set(1,data);
+ context.write(ckey,cvalue);
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.join.key.right"));
+ right_join_key_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.groupby.right"));
+ right_groupby_fnc = functional_argument(conf,code);
+ m = conf.getInt("mrql.m",1);
+ n = conf.getInt("mrql.n",1);
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the right mapper plan");
+ }
+ }
+ }
+
+ /** the GroupByJoin reducer */
+ private static class JoinReducer extends Reducer<GroupByJoinKey,MRContainer,MRContainer,MRContainer> {
+ private static String counter; // a Hadoop user-defined counter used in the repeat operation
+ private static int n, m; // n*m partitioners
+ private static Function left_groupby_fnc; // left group-by function
+ private static Function right_groupby_fnc;// right group-by function
+ private static Function map_fnc; // the map function
+ private static Function combine_fnc; // the combine function
+ private static Function reduce_fnc; // the reduce function
+ private static Bag left = new Bag(); // a cached bag of input fragments from left input
+ private static int current_partition = -1;
+ private static Hashtable<MRData,MRData> hashTable; // in-reducer combiner
+ private static Tuple pair = new Tuple(2);
+ private static MRContainer ckey = new MRContainer(new MR_int(0));
+ private static MRContainer cvalue = new MRContainer(new MR_int(0));
+ private static MRContainer container = new MRContainer(new MR_int(0));
+ private static Tuple tkey = new Tuple(2);
+ private static Bag tbag = new Bag(2);
+
+ private static void write ( MRContainer key, MRData value, Context context )
+ throws IOException, InterruptedException {
+ if (counter.equals("-")) {
+ container.set(value);
+ context.write(key,container);
+ } else { // increment the repetition counter if the repeat condition is true
+ Tuple t = (Tuple)value;
+ if (((MR_bool)t.second()).get())
+ context.getCounter("mrql",counter).increment(1);
+ container.set(t.first());
+ context.write(key,container);
+ }
+ }
+
+ private void store ( MRData key, MRData value ) throws IOException {
+ MRData old = hashTable.get(key);
+ Tuple k = (Tuple)key;
+ pair.set(0,key);
+ for ( MRData e: (Bag)map_fnc.eval(value) )
+ if (old == null)
+ hashTable.put(key,e);
+ else {
+ tbag.clear();
+ tbag.add_element(e).add_element(old);
+ pair.set(1,tbag);
+ for ( MRData z: (Bag)combine_fnc.eval(pair) )
+ hashTable.put(key,z); // normally, done once
+ }
+ }
+
+ protected static void flush_table ( Context context ) throws IOException, InterruptedException {
+ Enumeration<MRData> en = hashTable.keys();
+ while (en.hasMoreElements()) {
+ MRData key = en.nextElement();
+ MRData value = hashTable.get(key);
+ ckey.set(key);
+ pair.set(0,key);
+ tbag.clear();
+ tbag.add_element(value);
+ pair.set(1,tbag);
+ for ( MRData e: (Bag)reduce_fnc.eval(pair) )
+ write(ckey,e,context);
+ };
+ hashTable.clear();
+ }
+
+ @Override
+ public void reduce ( GroupByJoinKey key, Iterable<MRContainer> values, Context context )
+ throws IOException, InterruptedException {
+ if (key.partition != current_partition && current_partition > 0) {
+ // at the end of a partition, flush the hash table
+ flush_table(context);
+ current_partition = key.partition;
+ };
+ left.clear();
+ Tuple p = null;
+ final Iterator<MRContainer> i = values.iterator();
+ // left tuples arrive before right tuples; cache the left values into the left bag
+ while (i.hasNext()) {
+ p = (Tuple)i.next().data();
+ if (((MR_byte)p.first()).get() == 2)
+ break;
+ left.add(p.second());
+ p = null;
+ };
+ // the previous value was from right
+ if (p != null) {
+ MRData y = p.second();
+ MRData gy = right_groupby_fnc.eval(y);
+ // cross product with left (must use new Tuples)
+ for ( MRData x: left )
+ store(new Tuple(left_groupby_fnc.eval(x),gy),new Tuple(x,y));
+ // the rest of values are from right
+ while (i.hasNext()) {
+ y = ((Tuple)i.next().data()).second();
+ gy = right_groupby_fnc.eval(y);
+ // cross product with left (must use new Tuples)
+ for ( MRData x: left )
+ store(new Tuple(left_groupby_fnc.eval(x),gy),new Tuple(x,y));
+ }
+ }
+ }
+
+ @Override
+ protected void setup ( Context context ) throws IOException,InterruptedException {
+ super.setup(context);
+ try {
+ Configuration conf = context.getConfiguration();
+ Config.read(conf);
+ if (Plan.conf == null)
+ Plan.conf = conf;
+ Tree code = Tree.parse(conf.get("mrql.groupby.left"));
+ left_groupby_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.groupby.right"));
+ right_groupby_fnc = functional_argument(conf,code);
+ m = conf.getInt("mrql.m",1);
+ n = conf.getInt("mrql.n",1);
+ code = Tree.parse(conf.get("mrql.mapper"));
+ map_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.combiner"));
+ combine_fnc = functional_argument(conf,code);
+ code = Tree.parse(conf.get("mrql.reducer"));
+ reduce_fnc = functional_argument(conf,code);
+ counter = conf.get("mrql.counter");
+ hashTable = new Hashtable<MRData,MRData>(Config.map_cache_size);
+ } catch (Exception e) {
+ throw new Error("Cannot retrieve the reducer plan");
+ }
+ }
+
+ @Override
+ protected void cleanup ( Context context ) throws IOException,InterruptedException {
+ if (hashTable != null)
+ flush_table(context);
+ hashTable = null; // garbage-collect it
+ super.cleanup(context);
+ }
+ }
+
+ /** the GroupByJoin operation
+ * @param left_join_key_fnc left join key function
+ * @param right_join_key_fnc right join key function
+ * @param left_groupby_fnc left group-by function
+ * @param right_groupby_fnc right group-by function
+ * @param map_fnc map function
+ * @param combine_fnc combine function
+ * @param reduce_fnc reduce function
+ * @param X left data set
+ * @param Y right data set
+ * @param num_reducers number of reducers
+ * @param n left dimension of the reducer grid
+ * @param m right dimension of the reducer grid
+ * @param stop_counter optional counter used in repeat operation
+ * @return a DataSet that contains the result
+ */
+ public final static DataSet groupByJoin
+ ( Tree left_join_key_fnc, // left join key function
+ Tree right_join_key_fnc, // right join key function
+ Tree left_groupby_fnc, // left group-by function
+ Tree right_groupby_fnc, // right group-by function
+ Tree map_fnc, // map function
+ Tree combine_fnc, // combine function
+ Tree reduce_fnc, // reduce function
+ DataSet X, // left data set
+ DataSet Y, // right data set
+ int num_reducers, // number of reducers
+ int n, int m, // dimensions of the reducer grid
+ String stop_counter ) // optional counter used in repeat operation
+ throws Exception {
+ String newpath = new_path(conf);
+ conf.set("mrql.join.key.left",left_join_key_fnc.toString());
+ conf.set("mrql.join.key.right",right_join_key_fnc.toString());
+ conf.set("mrql.groupby.left",left_groupby_fnc.toString());
+ conf.set("mrql.groupby.right",right_groupby_fnc.toString());
+ conf.setInt("mrql.m",m);
+ conf.setInt("mrql.n",n);
+ conf.set("mrql.mapper",map_fnc.toString());
+ conf.set("mrql.combiner",combine_fnc.toString());
+ conf.set("mrql.reducer",reduce_fnc.toString());
+ conf.set("mrql.counter",stop_counter);
+ Job job = new Job(conf,newpath);
+ distribute_compiled_arguments(job.getConfiguration());
+ job.setMapOutputKeyClass(GroupByJoinKey.class);
+ job.setJarByClass(GroupByJoinPlan.class);
+ job.setOutputKeyClass(MRContainer.class);
+ job.setOutputValueClass(MRContainer.class);
+ job.setPartitionerClass(GroupByJoinPartitioner.class);
+ job.setSortComparatorClass(GroupByJoinSortComparator.class);
+ job.setGroupingComparatorClass(GroupByJoinGroupingComparator.class);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ FileOutputFormat.setOutputPath(job,new Path(newpath));
+ for (DataSource p: X.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperLeft.class);
+ for (DataSource p: Y.source)
+ MultipleInputs.addInputPath(job,new Path(p.path),(Class<? extends MapReduceMRQLFileInputFormat>)p.inputFormat,MapperRight.class);
+ job.setReducerClass(JoinReducer.class);
+ if (num_reducers > 0)
+ job.setNumReduceTasks(num_reducers);
+ job.waitForCompletion(true);
+ long c = (stop_counter.equals("-")) ? 0
+ : job.getCounters().findCounter("mrql",stop_counter).getValue();
+ DataSource s = new BinaryDataSource(newpath,conf);
+ s.to_be_merged = false;
+ return new DataSet(s,c,MapReducePlan.outputRecords(job));
+ }
+}
[08/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MRQLFileInputFormat.java b/src/main/java/core/MRQLFileInputFormat.java
deleted file mode 100644
index 6b6aec2..0000000
--- a/src/main/java/core/MRQLFileInputFormat.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-
-
-/** An interface for all MRQL FileInputFormats */
-public interface MRQLFileInputFormat {
- /** materialize the input file into a memory Bag */
- public Bag materialize ( final Path path ) throws IOException;
-
- /** materialize the entire dataset into a Bag
- * @param x the DataSet in HDFS to collect values from
- * @param strip is not used in MapReduce mode
- * @return the Bag that contains the collected values
- */
- public Bag collect ( final DataSet x, boolean strip ) throws Exception;
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_bool.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_bool.java b/src/main/java/core/MR_bool.java
deleted file mode 100644
index d40573f..0000000
--- a/src/main/java/core/MR_bool.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-/** a container for boolean values */
-final public class MR_bool extends MRData {
- private boolean value;
-
- public MR_bool ( boolean i ) { value = i; }
-
- public void materializeAll () {};
-
- public boolean get () { return value; }
-
- public void set ( boolean v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.BOOLEAN);
- out.writeBoolean(value);
- }
-
- final public static MR_bool read ( DataInput in ) throws IOException {
- return new MR_bool(in.readBoolean());
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = in.readBoolean();
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_bool);
- return (value == ((MR_bool) x).value) ? 0 : (value ? 1 : -1);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- size[0] = 2;
- boolean bx = x[xs] > 0;
- boolean by = y[ys] > 0;
- return (bx == by) ? 0 : (bx ? 1 : -1);
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_bool && ((MR_bool)x).value==value;
- }
-
- public int hashCode () { return value ? 0 : 1; }
-
- public String toString () {
- return Boolean.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_byte.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_byte.java b/src/main/java/core/MR_byte.java
deleted file mode 100644
index e5c319e..0000000
--- a/src/main/java/core/MR_byte.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for byte values */
-final public class MR_byte extends MRData {
- private byte value;
-
- public MR_byte ( byte i ) { value = i; }
- public MR_byte ( int i ) { value = (byte)i; }
-
- public void materializeAll () {};
-
- public byte get () { return value; }
-
- public void set ( byte v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.BYTE);
- out.writeByte(value);
- }
-
- final public static MR_byte read ( DataInput in ) throws IOException {
- return new MR_byte(in.readByte());
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = in.readByte();
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_byte);
- byte v = ((MR_byte) x).value;
- return (value == v) ? 0 : ((value < v) ? -1 : 1);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- size[0] = 2;
- return x[xs]-y[ys];
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_byte && ((MR_byte)x).value==value;
- }
-
- public int hashCode () { return Math.abs(value); }
-
- public String toString () {
- return Integer.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_char.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_char.java b/src/main/java/core/MR_char.java
deleted file mode 100644
index f0e9721..0000000
--- a/src/main/java/core/MR_char.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for a char */
-final public class MR_char extends MRData {
- private char value;
-
- public MR_char ( char i ) { value = i; }
-
- public void materializeAll () {};
-
- public char get () { return value; }
-
- public void set ( char v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.CHAR);
- out.writeChar(value);
- }
-
- final public static MR_char read ( DataInput in ) throws IOException {
- return new MR_char(in.readChar());
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = in.readChar();
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_char);
- return value-((MR_char) x).value;
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- size[0] = 2;
- return x[xs]-y[ys];
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_char && ((MR_char)x).value==value;
- }
-
- public int hashCode () { return value; }
-
- public String toString () {
- return Character.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_dataset.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_dataset.java b/src/main/java/core/MR_dataset.java
deleted file mode 100644
index abc81cd..0000000
--- a/src/main/java/core/MR_dataset.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a wrapper of a DataSet (stored in HDFS) as an MRData */
-final public class MR_dataset extends MRData {
- public DataSet dataset;
-
- public MR_dataset ( DataSet d ) { dataset = d; }
-
- public void materializeAll () {};
-
- public DataSet dataset () { return dataset; }
-
- public void write ( DataOutput out ) throws IOException {
- throw new Error("DataSets are not serializable");
- }
-
- public void readFields ( DataInput in ) throws IOException {
- throw new Error("DataSets are not serializable");
- }
-
- public int compareTo ( MRData x ) {
- throw new Error("DataSets cannot be compared");
- }
-
- public boolean equals ( Object x ) {
- throw new Error("DataSets cannot be compared");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_double.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_double.java b/src/main/java/core/MR_double.java
deleted file mode 100644
index 567917c..0000000
--- a/src/main/java/core/MR_double.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for double values */
-final public class MR_double extends MRData {
- private double value;
-
- public MR_double ( double x ) { value = x; }
-
- public void materializeAll () {};
-
- public double get () { return value; }
-
- public void set ( double v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.DOUBLE);
- out.writeDouble(value);
- }
-
- final public static MR_double read ( DataInput in ) throws IOException {
- return new MR_double(in.readDouble());
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = in.readDouble();
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_double);
- double v = ((MR_double) x).value;
- return (value == v) ? 0 : ((value > v) ? 1 : -1);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- size[0] = (Double.SIZE >> 3)+1;
- double v = WritableComparator.readDouble(x,xs) - WritableComparator.readDouble(y,ys);
- return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_double && ((MR_double)x).value==value;
- }
-
- public int hashCode () {
- return Math.abs((int)Double.doubleToLongBits(value));
- }
-
- public String toString () {
- return Double.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_float.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_float.java b/src/main/java/core/MR_float.java
deleted file mode 100644
index d8dda38..0000000
--- a/src/main/java/core/MR_float.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for float values */
-final public class MR_float extends MRData {
- private float value;
-
- public MR_float ( float x ) { value = x; }
-
- public MR_float ( double x ) { value = (float)x; }
-
- public void materializeAll () {};
-
- public float get () { return value; }
-
- public void set ( float v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.FLOAT);
- out.writeFloat(value);
- }
-
- final public static MR_float read ( DataInput in ) throws IOException {
- return new MR_float(in.readFloat());
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = in.readFloat();
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_float);
- float v = ((MR_float) x).value;
- return (value == v) ? 0 : ((value > v) ? 1 : -1);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- size[0] = (Float.SIZE >> 3)+1;
- float v = WritableComparator.readFloat(x,xs) - WritableComparator.readFloat(y,ys);
- return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_float && ((MR_float)x).value==value;
- }
-
- public int hashCode () {
- return Math.abs(Float.floatToIntBits(value));
- }
-
- public String toString () {
- return Float.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_int.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_int.java b/src/main/java/core/MR_int.java
deleted file mode 100644
index 6586fe0..0000000
--- a/src/main/java/core/MR_int.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for int values */
-final public class MR_int extends MRData {
- private int value;
-
- public MR_int ( int i ) { value = i; }
-
- public void materializeAll () {};
-
- public int get () { return value; }
-
- public void set ( int v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.INT);
- WritableUtils.writeVInt(out,value);
- }
-
- final public static MR_int read ( DataInput in ) throws IOException {
- return new MR_int(WritableUtils.readVInt(in));
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = WritableUtils.readVInt(in);
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_int);
- return value - ((MR_int) x).value;
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- try {
- size[0] = 1+WritableUtils.decodeVIntSize(x[xs]);
- int v = WritableComparator.readVInt(x,xs)-WritableComparator.readVInt(y,ys);
- return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_int && ((MR_int)x).value==value;
- }
-
- public int hashCode () { return Math.abs(value); }
-
- public String toString () {
- return Integer.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_long.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_long.java b/src/main/java/core/MR_long.java
deleted file mode 100644
index 2078048..0000000
--- a/src/main/java/core/MR_long.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for long values */
-final public class MR_long extends MRData {
- private long value;
-
- public MR_long ( long i ) { value = i; }
-
- public void materializeAll () {};
-
- public long get () { return value; }
-
- public void set ( long v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.LONG);
- WritableUtils.writeVLong(out,value);
- }
-
- final public static MR_long read ( DataInput in ) throws IOException {
- return new MR_long(WritableUtils.readVLong(in));
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = WritableUtils.readVLong(in);
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_long);
- long v = ((MR_long) x).value;
- return (value == v) ? 0 : ((value > v) ? 1 : -1);
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- try {
- size[0] = 1+WritableUtils.decodeVIntSize(x[xs]);
- long v = WritableComparator.readVLong(x,xs)-WritableComparator.readVLong(y,ys);
- return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_long && ((MR_long)x).value==value;
- }
-
- public int hashCode () { return (int)Math.abs(value); }
-
- public String toString () {
- return Long.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_more_bsp_steps.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_more_bsp_steps.java b/src/main/java/core/MR_more_bsp_steps.java
deleted file mode 100644
index 0d844ad..0000000
--- a/src/main/java/core/MR_more_bsp_steps.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-
-
-/** used for BSP synchronization when a peer needs to do more steps */
-final public class MR_more_bsp_steps extends MRData {
- MR_more_bsp_steps () {}
-
- public void materializeAll () {};
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.MORE_BSP_STEPS);
- }
-
- public void readFields ( DataInput in ) throws IOException {}
-
- public int compareTo ( MRData x ) { return 0; }
-
- public boolean equals ( Object x ) { return x instanceof MR_more_bsp_steps; }
-
- public int hashCode () { return 0; }
-
- public String toString () { return "more_bsp_steps"; }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_short.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_short.java b/src/main/java/core/MR_short.java
deleted file mode 100644
index c50e124..0000000
--- a/src/main/java/core/MR_short.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for short values */
-final public class MR_short extends MRData {
- private short value;
-
- public MR_short ( short i ) { value = i; }
-
- public void materializeAll () {};
-
- public short get () { return value; }
-
- public void set ( short v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.SHORT);
- out.writeShort(value);
- }
-
- final public static MR_short read ( DataInput in ) throws IOException {
- return new MR_short(in.readShort());
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = in.readShort();
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_short);
- return value - ((MR_short) x).value;
- }
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- size[0] = (Short.SIZE >> 3)+1;
- return WritableComparator.readUnsignedShort(x,xs) - WritableComparator.readUnsignedShort(y,ys);
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_short && ((MR_short)x).value==value;
- }
-
- public int hashCode () { return Math.abs((int)value); }
-
- public String toString () {
- return Short.toString(value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_string.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_string.java b/src/main/java/core/MR_string.java
deleted file mode 100644
index a914cca..0000000
--- a/src/main/java/core/MR_string.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/** a container for strings */
-final public class MR_string extends MRData {
- private String value;
-
- public MR_string ( String s ) { value = s; }
-
- public void materializeAll () {};
-
- public String get () { return value; }
-
- public void set ( String v ) { value = v; }
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.STRING);
- Text.writeString(out,value);
- }
-
- final public static MR_string read ( DataInput in ) throws IOException {
- return new MR_string(Text.readString(in));
- }
-
- public void readFields ( DataInput in ) throws IOException {
- value = Text.readString(in);
- }
-
- public int compareTo ( MRData x ) {
- assert(x instanceof MR_string);
- return value.compareTo(((MR_string) x).value);
- }
-
- final static Text.Comparator comparator = new Text.Comparator();
-
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- try {
- size[0] = 1+WritableComparator.readVInt(x,xs)+WritableUtils.decodeVIntSize(x[xs]);
- return comparator.compare(x,xs,xl,y,ys,yl);
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- public boolean equals ( Object x ) {
- return x instanceof MR_string && value.equals(((MR_string) x).value);
- }
-
- public int hashCode () { return value.hashCode(); }
-
- public String toString () {
- return "\""+value+"\"";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_sync.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_sync.java b/src/main/java/core/MR_sync.java
deleted file mode 100644
index 275414f..0000000
--- a/src/main/java/core/MR_sync.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-
-
-/** used for BSP synchronization */
-final public class MR_sync extends MRData {
- MR_sync () {}
-
- public void materializeAll () {};
-
- final public void write ( DataOutput out ) throws IOException {
- out.writeByte(MRContainer.SYNC);
- }
-
- public void readFields ( DataInput in ) throws IOException {}
-
- public int compareTo ( MRData x ) { return 0; }
-
- public boolean equals ( Object x ) { return x instanceof MR_sync; }
-
- public int hashCode () { return 0; }
-
- public String toString () { return "sync"; }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MR_variable.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MR_variable.java b/src/main/java/core/MR_variable.java
deleted file mode 100644
index f81f117..0000000
--- a/src/main/java/core/MR_variable.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-
-
-/** a template variable; should appear only in a template */
-final public class MR_variable extends MRData {
- public int var_num;
-
- MR_variable ( int n ) { var_num = n; }
-
- public void materializeAll () {};
-
- final public void write ( DataOutput out ) throws IOException {}
-
- public void readFields ( DataInput in ) throws IOException {}
-
- public int compareTo ( MRData x ) { return 0; }
-
- public boolean equals ( Object x ) { return false; }
-
- public int hashCode () { return 0; }
-
- public String toString () {
- return "variable("+var_num+")";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Main.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Main.java b/src/main/java/core/Main.java
deleted file mode 100644
index 6e0f527..0000000
--- a/src/main/java/core/Main.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import java.util.Enumeration;
-import org.apache.hadoop.util.*;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.log4j.*;
-import jline.*;
-
-
-final public class Main {
- public final static String version = "0.9.0";
-
- public static PrintStream print_stream;
- public static Configuration conf;
- static MRQLParser parser = new MRQLParser();
- public static String query_file = "";
-
- public static void include_file ( String file ) {
- try {
- MRQLParser old_parser = parser;
- boolean old_interactive = Config.interactive;
- try {
- parser = new MRQLParser(new MRQLLex(new FileInputStream(file)));
- } catch (Exception e) {
- Path path = new Path(file);
- FileSystem fs = path.getFileSystem(conf);
- parser = new MRQLParser(new MRQLLex(fs.open(path)));
- };
- Config.interactive = false;
- parser.parse();
- Config.interactive = old_interactive;
- parser = old_parser;
- } catch (Exception ex) {
- ex.printStackTrace(System.err);
- throw new Error(ex);
- }
- }
-
- public static void main ( String[] args ) throws Exception {
- boolean hadoop = false;
- for ( String arg: args ) {
- hadoop |= arg.equals("-local") || arg.equals("-dist");
- Config.bsp_mode |= arg.equals("-bsp");
- Config.spark_mode |= arg.equals("-spark");
- };
- Config.map_reduce_mode = !Config.bsp_mode && !Config.spark_mode;
- if (Config.map_reduce_mode)
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.MapReduceEvaluator").newInstance();
- if (Config.bsp_mode)
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
- if (Config.spark_mode)
- Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.SparkEvaluator").newInstance();
- if (hadoop) {
- conf = Evaluator.evaluator.new_configuration();
- GenericOptionsParser gop = new GenericOptionsParser(conf,args);
- conf = gop.getConfiguration();
- args = gop.getRemainingArgs();
- };
- Config.parse_args(args,conf);
- Config.hadoop_mode = Config.local_mode || Config.distributed_mode;
- if (!Config.info) {
- for ( Enumeration en = LogManager.getCurrentLoggers(); en.hasMoreElements(); )
- ((Logger)en.nextElement()).setLevel(Level.WARN);
- LogManager.getRootLogger().setLevel(Level.WARN);
- };
- Evaluator.evaluator.init(conf);
- new TopLevel();
- System.out.print("Apache MRQL version "+version+" (");
- if (Config.compile_functional_arguments)
- System.out.print("compiled ");
- else System.out.print("interpreted ");
- if (hadoop) {
- if (Config.local_mode)
- System.out.print("local ");
- else if (Config.distributed_mode)
- System.out.print("distributed ");
- if (Config.spark_mode)
- System.out.println("Spark mode using "+Config.nodes+" tasks)");
- else if (Config.bsp_mode)
- System.out.println("Hama BSP mode over "+Config.nodes+" BSP tasks)");
- else if (Config.nodes > 0)
- System.out.println("Hadoop MapReduce mode with "+Config.nodes+" reducers)");
- else if (!Config.local_mode)
- System.out.println("Hadoop MapReduce mode with 1 reducer, use -nodes to change it)");
- else System.out.println("Hadoop MapReduce mode)");
- } else if (Config.bsp_mode)
- System.out.println("in-memory BSP mode)");
- else System.out.println("in-memory MapReduce mode)");
- if (Config.interactive) {
- System.out.println("Type quit to exit");
- ConsoleReader reader = new ConsoleReader();
- reader.setBellEnabled(false);
- History history = new History(new File(System.getProperty("user.home")+"/.mrqlhistory"));
- reader.setHistory(history);
- reader.setUseHistory(false);
- try {
- loop: while (true) {
- String line = "";
- String s = "";
- try {
- if (hadoop && Config.bsp_mode)
- Config.write(Plan.conf);
- do {
- s = reader.readLine("> ");
- if (s != null && (s.equals("quit") || s.equals("exit")))
- break loop;
- if (s != null)
- line += " "+s;
- } while (s == null || s.indexOf(";") <= 0);
- line = line.substring(1);
- history.addToHistory(line);
- parser = new MRQLParser(new MRQLLex(new StringReader(line)));
- MRQLLex.reset();
- parser.parse();
- } catch (EOFException x) {
- break;
- } catch (Exception x) {
- if (x.getMessage() != null)
- System.out.println(x);
- } catch (Error x) {
- System.out.println(x);
- }
- }
- } finally {
- if (hadoop) {
- Plan.clean();
- Evaluator.evaluator.shutdown(Plan.conf);
- };
- if (Config.compile_functional_arguments)
- Compiler.clean();
- }
- } else try {
- if (hadoop && Config.bsp_mode)
- Config.write(Plan.conf);
- try {
- parser = new MRQLParser(new MRQLLex(new FileInputStream(query_file)));
- } catch (Exception e) {
- // when the query file is in HDFS
- Path path = new Path(query_file);
- FileSystem fs = path.getFileSystem(conf);
- parser = new MRQLParser(new MRQLLex(fs.open(path)));
- };
- parser.parse();
- } finally {
- if (hadoop) {
- Plan.clean();
- Evaluator.evaluator.shutdown(Plan.conf);
- };
- if (Config.compile_functional_arguments)
- Compiler.clean();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MapReduceAlgebra.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MapReduceAlgebra.java b/src/main/java/core/MapReduceAlgebra.java
deleted file mode 100644
index 198a533..0000000
--- a/src/main/java/core/MapReduceAlgebra.java
+++ /dev/null
@@ -1,810 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.*;
-
-
-/** Evaluation of MRQL algebra expressions in memory */
-final public class MapReduceAlgebra {
-
- /** eager concat-map (not used) */
- private static Bag cmap_eager ( final Function f, final Bag s ) {
- Bag res = new Bag();
- for ( MRData e: s )
- res.addAll((Bag)f.eval(e));
- return res;
- }
-
- /** lazy concat-map (stream-based)
- * @param f a function from a to {b}
- * @param s the input of type {a}
- * @return a value of type {b}
- */
- public static Bag cmap ( final Function f, final Bag s ) {
- final Iterator<MRData> si = s.iterator();
- return new Bag(new BagIterator() {
- Iterator<MRData> data = null;
- boolean more = false;
- public boolean hasNext () {
- if (data == null) {
- while (!more && si.hasNext()) {
- data = ((Bag)f.eval(si.next())).iterator();
- more = data.hasNext();
- }
- } else {
- if (more) {
- more = data.hasNext();
- if (more)
- return true;
- };
- while (!more && si.hasNext()) {
- data = ((Bag)f.eval(si.next())).iterator();
- more = data.hasNext();
- }
- };
- return more;
- }
- public MRData next () {
- return data.next();
- }
- });
- }
-
- /** lazy map
- * @param f a function from a to b
- * @param s the input of type {a}
- * @return a value of type {b}
- */
- public static Bag map ( final Function f, final Bag s ) {
- final Iterator<MRData> si = s.iterator();
- return new Bag(new BagIterator() {
- public boolean hasNext () { return si.hasNext(); }
- public MRData next () { return f.eval(si.next()); }
- });
- }
-
- /** lazy filter combined with a map
- * @param p a function from a to boolean
- * @param f a function from a to b
- * @param s the input of type {a}
- * @return a value of type {b}
- */
- public static Bag filter ( final Function p, final Function f, final Bag s ) {
- final Iterator<MRData> si = s.iterator();
- return new Bag(new BagIterator() {
- MRData data = null;
- public boolean hasNext () {
- while (si.hasNext()) {
- data = si.next();
- if (((MR_bool)p.eval(data)).get())
- return true;
- };
- return false;
- }
- public MRData next () { return f.eval(data); }
- });
- }
-
- /** strict group-by
- * @param s the input of type {(a,b)}
- * @return a value of type {(a,{b})}
- */
- public static Bag groupBy ( Bag s ) {
- Bag res = new Bag();
- s.sort();
- MRData last = null;
- Bag group = new Bag();
- for ( MRData e: s) {
- final Tuple p = (Tuple)e;
- if (last != null && p.first().equals(last))
- group.add(p.second());
- else {
- if (last != null) {
- group.trim();
- res.add(new Tuple(last,group));
- };
- last = p.first();
- group = new Bag();
- group.add(p.second());
- }
- };
- if (last != null) {
- group.trim();
- res.add(new Tuple(last,group));
- };
- //res.trim();
- return res;
- }
-
- /** lazy group-by (not used) */
- private static Bag groupBy_lazy ( Bag s ) {
- s.sort();
- final Iterator<MRData> it = s.iterator();
- return new Bag(new BagIterator() {
- MRData last = null;
- MRData data = null;
- Bag group = new Bag();
- public boolean hasNext () {
- while (it.hasNext()) {
- final Tuple p = (Tuple)it.next();
- if (last != null && p.first().equals(last))
- group.add(p.second());
- else if (last != null) {
- group.trim();
- data = new Tuple(last,group);
- last = p.first();
- group = new Bag();
- group.add(p.second());
- return true;
- } else {
- last = p.first();
- group = new Bag();
- group.add(p.second());
- }
- };
- if (last != null) {
- group.trim();
- data = new Tuple(last,group);
- last = null;
- return true;
- };
- return false;
- }
- public MRData next () {
- return data;
- }
- });
- }
-
- /** the MapReduce operation
- * @param m a map function from a to {(k,b)}
- * @param r a reduce function from (k,{b}) to {c}
- * @param s the input of type {a}
- * @return a value of type {c}
- */
- public static Bag mapReduce ( final Function m, final Function r, final Bag s ) {
- return cmap(r,groupBy(cmap(m,s)));
- }
-
- /** Not used: use mapReduce2 instead */
- private static Bag join ( final Function kx, final Function ky, final Function f,
- final Bag X, final Bag Y ) {
- return cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple p = (Tuple)e;
- return (Bag)f.eval(new Tuple(p.second(),
- cmap(new Function() {
- public Bag eval ( final MRData y ) {
- return (ky.eval(y).equals(p.first()))
- ? new Bag(y)
- : new Bag();
- } }, Y))); }
- },
- groupBy(cmap(new Function() {
- public Bag eval ( final MRData x ) {
- return new Bag(new Tuple(kx.eval(x),x));
- } }, X)));
- }
-
- /** A hash-based equi-join
- * @param kx left key function from a to k
- * @param ky right key function from b to k
- * @param f reducer from (a,b) to c
- * @param X left input of type {a}
- * @param Y right input of type {b}
- * @return a value of type {c}
- */
- public static Bag hash_join ( final Function kx, final Function ky, final Function f,
- final Bag X, final Bag Y ) {
- Hashtable<MRData,Bag> hashTable = new Hashtable<MRData,Bag>(1000);
- for ( MRData x: X ) {
- MRData key = kx.eval(x);
- Bag old = hashTable.get(key);
- if (old == null)
- hashTable.put(key,new Bag(x));
- else old.add(x);
- };
- Bag res = new Bag();
- for ( MRData y: Y ) {
- MRData key = ky.eval(y);
- Bag match = hashTable.get(key);
- if (match != null)
- for ( MRData x: match )
- res.add(f.eval(new Tuple(x,y)));
- };
- return res;
- }
-
- /** A cross-product
- * @param mx left map function from a to {a'}
- * @param my right key function from b to {b'}
- * @param r reducer from (a',b') to {c}
- * @param X left input of type {a}
- * @param Y right input of type {b}
- * @return a value of type {c}
- */
- public static Bag crossProduct ( final Function mx, final Function my, final Function r,
- final Bag X, final Bag Y ) {
- Bag a = new Bag();
- for ( MRData y: Y )
- for ( MRData v: (Bag)my.eval(y) )
- a.add(v);
- Bag b = new Bag();
- for ( MRData x: X )
- for ( MRData xx: (Bag)mx.eval(x) )
- for ( MRData y: a )
- for ( MRData v: (Bag)r.eval(new Tuple(xx,y)) )
- b.add(v);
- return b;
- }
-
- /** A map-reduce operation with two mappers (a join)
- * @param mx left map function from a to {(k,a')}
- * @param my right key function from b to {(k,b')}
- * @param r reducer from ({a'},{b'}) to {c}
- * @param X left input of type {a}
- * @param Y right input of type {b}
- * @return a value of type {c}
- */
- public static Bag mapReduce2 ( final Function mx, // left mapper
- final Function my, // right mapper
- final Function r, // reducer
- final Bag X, final Bag Y ) {
- final Bag left = cmap(new Function() {
- public Bag eval ( final MRData x ) {
- return cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple p = (Tuple)e;
- return new Bag(new Tuple(p.first(),
- new Tuple(new MR_byte(1),p.second())));
- } }, (Bag)mx.eval(x));
- } }, X);
- final Bag right = cmap(new Function() {
- public Bag eval ( final MRData y ) {
- return cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple p = (Tuple)e;
- return new Bag(new Tuple(p.first(),
- new Tuple(new MR_byte(2),p.second())));
- } }, (Bag)my.eval(y));
- } }, Y);
- final Iterator<MRData> li = left.iterator();
- final Iterator<MRData> ri = right.iterator();
- final Bag mix = new Bag(new BagIterator () {
- MRData data;
- public boolean hasNext () {
- if (li.hasNext()) {
- data = li.next();
- return true;
- } else if (ri.hasNext()) {
- data = ri.next();
- return true;
- } else return false;
- }
- public MRData next () {
- return data;
- }
- });
- return cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple p = (Tuple)e;
- final Bag xs = cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple q = (Tuple)e;
- return (((MR_byte)q.first()).get() == 1)
- ? new Bag(q.second())
- : new Bag();
- } }, (Bag)p.second());
- final Bag ys = cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple q = (Tuple)e;
- return (((MR_byte)q.first()).get() == 2)
- ? new Bag(q.second())
- : new Bag();
- } }, (Bag)p.second());
- xs.materialize();
- ys.materialize();
- return (Bag)r.eval(new Tuple(xs,ys));
- } }, groupBy(mix));
- }
-
- /** The fragment-replicate join (map-side join)
- * @param kx left key function from a to k
- * @param ky right key function from b to k
- * @param r reducer from (a,{b}) to {c}
- * @param X left input of type {a}
- * @param Y right input of type {b}
- * @return a value of type {c}
- */
- public static Bag mapJoin ( final Function kx, final Function ky, final Function r,
- final Bag X, final Bag Y ) {
- X.materialize();
- Y.materialize();
- return cmap(new Function() {
- public Bag eval ( final MRData e ) {
- final Tuple p = (Tuple)e;
- return cmap(new Function() {
- public Bag eval ( final MRData x ) {
- return (kx.eval(x).equals(p.first()))
- ? (Bag)r.eval(new Tuple(x,p.second()))
- : new Bag();
- } }, X); }
- },
- groupBy(cmap(new Function() {
- public Bag eval ( final MRData y ) {
- return new Bag(new Tuple(ky.eval(y),y));
- } }, Y)));
- }
-
- /** An equi-join combined with a group-by (see GroupByJoinPlan)
- * @param kx left key function from a to k
- * @param ky right key function from b to k
- * @param gx group-by key function from a to k1
- * @param gy group-by key function from b to k2
- * @param m mapper from (a,b) to {c}
- * @param c combiner from ((k1,k2),{c}) to d
- * @param r reducer from ((k1,k2),d) to {e}
- * @param X left input of type {a}
- * @param Y right input of type {b}
- * @return a value of type {e}
- */
- public static Bag groupByJoin ( final Function kx, final Function ky,
- final Function gx, final Function gy,
- final Function m, final Function c, final Function r,
- final Bag X, final Bag Y ) {
- Bag s = groupBy(hash_join(kx,ky,
- new Function() {
- public MRData eval ( final MRData e ) {
- Tuple t = (Tuple)e;
- return new Tuple(new Tuple(gx.eval(t.first()),gy.eval(t.second())),t);
- } },
- X,Y));
- Bag res = new Bag();
- for ( MRData z: s ) {
- Tuple t = (Tuple)z;
- for ( MRData n: (Bag)r.eval(new Tuple(t.first(),c.eval(new Tuple(t.first(),cmap(m,(Bag)t.second()))))) )
- res.add(n);
- };
- return res;
- }
-
- private static void flush_table ( final Map<MRData,MRData> hashTable, final Function r, final Bag result ) {
- Bag tbag = new Bag(2);
- Tuple pair = new Tuple(2);
- for ( Map.Entry<MRData,MRData> entry: hashTable.entrySet() ) {
- pair.set(0,entry.getKey());
- tbag.clear();
- tbag.add_element(entry.getValue());
- pair.set(1,tbag);
- for ( MRData e: (Bag)r.eval(pair) )
- result.add(e);
- };
- hashTable.clear();
- }
-
- /** An equi-join combined with a group-by implemented using hashing
- * @param kx left key function from a to k
- * @param ky right key function from b to k
- * @param gx group-by key function from a to k1
- * @param gy group-by key function from b to k2
- * @param m mapper from (a,b) to {c}
- * @param c combiner from ((k1,k2),{c}) to d
- * @param r reducer from ((k1,k2),d) to {e}
- * @param X left input of type {a}
- * @param Y right input of type {b}
- * @return a value of type {e}
- */
- final public static Bag mergeGroupByJoin ( final Function kx, final Function ky,
- final Function gx, final Function gy,
- final Function m, final Function c, final Function r,
- Bag X, Bag Y ) {
- Bag tbag = new Bag(2);
- Tuple pair = new Tuple(2);
- Tuple vpair = new Tuple(2);
- final Map<MRData,MRData> hashTable = new HashMap<MRData,MRData>(1000);
- Bag xs = groupBy(map(new Function() {
- public MRData eval ( final MRData e ) {
- Tuple t = (Tuple)e;
- return new Tuple(new Tuple(t.first(),kx.eval(t.second())),t.second());
- } }, X));
- Bag ys = groupBy(map(new Function() {
- public MRData eval ( final MRData e ) {
- Tuple t = (Tuple)e;
- return new Tuple(new Tuple(t.first(),ky.eval(t.second())),t.second());
- } }, Y));
- X = null; Y = null;
- Bag res = new Bag();
- final Iterator<MRData> xi = xs.iterator();
- final Iterator<MRData> yi = ys.iterator();
- if ( !xi.hasNext() || !yi.hasNext() )
- return res;
- Tuple x = (Tuple)xi.next();
- Tuple y = (Tuple)yi.next();
- MRData partition = null;
- while ( xi.hasNext() && yi.hasNext() ) {
- int cmp = x.first().compareTo(y.first());
- if (cmp < 0) { x = (Tuple)xi.next(); continue; };
- if (cmp > 0) { y = (Tuple)yi.next(); continue; };
- if (partition == null)
- partition = ((Tuple)x.first()).first();
- else if (!partition.equals(((Tuple)x.first()).first())) {
- partition = ((Tuple)x.first()).first();
- flush_table(hashTable,r,res);
- };
- for ( MRData xx: (Bag)x.second() )
- for ( MRData yy: (Bag)y.second() ) {
- Tuple key = new Tuple(gx.eval(xx),gy.eval(yy));
- vpair.set(0,xx).set(1,yy);
- MRData old = hashTable.get(key);
- pair.set(0,key);
- for ( MRData e: (Bag)m.eval(vpair) )
- if (old == null)
- hashTable.put(key,e);
- else {
- tbag.clear();
- tbag.add_element(e).add_element(old);
- pair.set(1,tbag);
- for ( MRData z: (Bag)c.eval(pair) )
- hashTable.put(key,z); // normally, done once
- }
- };
- if (xi.hasNext())
- x = (Tuple)xi.next();
- if (yi.hasNext())
- y = (Tuple)yi.next();
- };
- flush_table(hashTable,r,res);
- return res;
- }
-
- /** repeat the loop until all termination conditions are true or until we reach the max num of steps
- * @param loop a function from {a} to {(a,boolean)}
- * @param init the initial value of type {a}
- * @param max_num the maximum number of steps
- * @return a value of type {a}
- */
- public static Bag repeat ( final Function loop,
- final Bag init,
- final int max_num ) throws Exception {
- boolean cont;
- int i = 0;
- Bag s = init;
- s.materializeAll();
- do {
- MRData d = loop.eval(s);
- i++;
- cont = false;
- if (d instanceof Bag) {
- Bag bag = (Bag) d;
- bag.materialize();
- s.clear();
- for ( MRData x: bag ) {
- Tuple t = (Tuple)x;
- cont |= ((MR_bool)t.second()).get();
- s.add(t.first());
- }
- } else if (d instanceof MR_dataset) {
- DataSet ds = ((MR_dataset)d).dataset();
- if (ds.counter != 0)
- cont = true;
- System.err.println("*** Repeat #"+i+": "+ds.counter+" true results");
- s = Plan.collect(ds);
- } else throw new Error("Wrong repeat");
- } while (cont && i <= max_num);
- return s;
- }
-
- /** transitive closure: repeat the loop until the new set is equal to the previous set
- * or until we reach the max num of steps
- * @param loop a function from {a} to {a}
- * @param init the initial value of type {a}
- * @param max_num the maximum number of steps
- * @return a value of type {a}
- */
- public static Bag closure ( final Function loop,
- final Bag init,
- final int max_num ) throws Exception {
- int i = 0;
- long n = 0;
- long old = 0;
- Bag s = init;
- s.materializeAll();
- do {
- MRData d = loop.eval(s);
- i++;
- if (d instanceof Bag) {
- s = (Bag)d;
- s.materialize();
- old = n;
- n = s.size();
- } else if (d instanceof MR_dataset) {
- DataSet ds = ((MR_dataset)d).dataset();
- System.err.println("*** Repeat #"+i+": "+(ds.records-n)+" new records");
- old = n;
- n = ds.records;
- s = Plan.collect(ds);
- } else throw new Error("Wrong repeat");
- } while (old < n && i <= max_num);
- return s;
- }
-
- /** parse a text document using a given parser
- * @param parser the parser
- * @param file the text document (local file)
- * @param args the arguments to pass to the parser
- * @return a lazy bag that contains the parsed data
- */
- public static Bag parsedSource ( final Parser parser,
- final String file,
- Trees args ) {
- try {
- parser.initialize(args);
- parser.open(file);
- return new Bag(new BagIterator() {
- Iterator<MRData> result = null;
- MRData data;
- public boolean hasNext () {
- try {
- while (result == null || !result.hasNext()) {
- String s = parser.slice();
- if (s == null)
- return false;
- result = parser.parse(s).iterator();
- };
- data = (MRData)result.next();
- return true;
- } catch (Exception e) {
- throw new Error(e);
- }
- }
- public MRData next () {
- return data;
- }
- });
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- /** parse a text document using a given parser
- * @param parser the name of the parser
- * @param file the text document (local file)
- * @param args the arguments to pass to the parser
- * @return a lazy bag that contains the parsed data
- */
- public static Bag parsedSource ( String parser, String file, Trees args ) {
- try {
- return parsedSource(DataSource.parserDirectory.get(parser).newInstance(),file,args);
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- private static Bag add_source_num ( int source_num, Bag input ) {
- return new Bag(new Tuple(new MR_int(source_num),input));
- }
-
- /** parse a text document using a given parser and tag output data with a source num
- * @param source_num the source id
- * @param parser the parser
- * @param file the text document (local file)
- * @param args the arguments to pass to the parser
- * @return a lazy bag that contains the parsed data taged with the source id
- */
- public static Bag parsedSource ( int source_num,
- Parser parser,
- String file,
- Trees args ) {
- return add_source_num(source_num,parsedSource(parser,file,args));
- }
-
- /** parse a text document using a given parser and tag output data with a source num
- * @param source_num the source id
- * @param parser the name of the parser
- * @param file the text document (local file)
- * @param args the arguments to pass to the parser
- * @return a lazy bag that contains the parsed data taged with the source id
- */
- public static Bag parsedSource ( int source_num, String parser, String file, Trees args ) {
- try {
- return parsedSource(source_num,DataSource.parserDirectory.get(parser).newInstance(),file,args);
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- /** aggregate the Bag elements
- * @param accumulator a function from (b,a) to b
- * @param zero a value of type b
- * @param s a Bag of type {a}
- * @return a value of type b
- */
- public static MRData aggregate ( final Function accumulator,
- final MRData zero,
- final Bag s ) {
- MRData result = zero;
- for ( MRData x: s )
- result = accumulator.eval(new Tuple(result,x));
- return result;
- }
-
- public static MRData materialize ( MRData x ) {
- if (x instanceof Bag)
- ((Bag)x).materialize();
- return x;
- }
-
- /** Dump the value of some type to a binary local file;
- * The type is dumped to a separate file.type
- */
- public static void dump ( String file, Tree type, MRData value ) throws IOException {
- PrintStream ftp = new PrintStream(file+".type");
- ftp.print("1@"+type.toString()+"\n");
- ftp.close();
- DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(file)));
- value.write(out);
- out.close();
- }
-
- /** return the type of the dumped binary local file from file.type */
- public static Tree get_type ( String file ) {
- try {
- BufferedReader ftp = new BufferedReader(new FileReader(new File(file+".type")));
- String s[] = ftp.readLine().split("@");
- ftp.close();
- if (s.length != 2)
- return null;
- if (!s[0].equals("1"))
- throw new Error("The binary file has been created in hadoop mode and cannot be read in java mode");
- return Tree.parse(s[1]);
- } catch (Exception e) {
- return null;
- }
- }
-
- /** read the contents of a dumped local binary file */
- public static MRData read_binary ( String file ) {
- try {
- Tree type = get_type(file);
- DataInputStream in = new DataInputStream(new FileInputStream(new File(file)));
- return MRContainer.read(in);
- } catch (Exception e) {
- return null;
- }
- }
-
- /** read the contents of a dumped local binary file and tag data with a source num */
- public static Bag read_binary ( int source_num, String file ) {
- return add_source_num(source_num,(Bag)read_binary(file));
- }
-
- /** generate a lazy bag of long numbers {min...max} */
- public static Bag generator ( final long min, final long max ) {
- if (min > max)
- throw new Error("Min value ("+min+") is larger than max ("+max+") in generator");
- return new Bag(new BagIterator() {
- long index = min;
- public boolean hasNext () {
- return index <= max;
- }
- public MRData next () {
- return new MR_long(index++);
- }
- });
- }
-
- /** generate a lazy bag of long numbers {min...max} and tag each lon number with a source num */
- public static Bag generator ( int source_num, final long min, final long max ) {
- return add_source_num(source_num,generator(min,max));
- }
-
- /** the cache that holds all local data in memory */
- private static Tuple cache;
-
- /** return the cache element at location loc */
- public static MRData getCache ( int loc ) {
- return cache.get(loc);
- }
-
- /** set the cache element at location loc to value and return ret */
- public static MRData setCache ( int loc, MRData value, MRData ret ) {
- if (value instanceof Bag)
- materialize((Bag)value);
- cache.set(loc,value);
- return ret;
- }
-
- /** The BSP operation
- * @param source the source ids of the input Bags
- * @param superstep the BSP superstep is a function from ({M},S) to ({M},S,boolean)
- * @param init_state is the initial state of type S
- * @param order do we need to order the result?
- * @param inputs the input Bags
- * @return return a Bag in cache[0]
- */
- public static MRData BSP ( final int[] source,
- final Function superstep,
- final MRData init_state,
- boolean order,
- final Bag[] inputs ) {
- Bag msgs = new Bag();
- MRData state = init_state;
- Tuple result;
- boolean exit;
- boolean skip = false;
- String tabs = "";
- int step = 0;
- cache = new Tuple(100);
- for ( int i = 0; i < 100; i++ )
- cache.set(i,new Bag());
- for ( Bag x: inputs ) {
- Tuple p = (Tuple)(x.get(0));
- cache.set(((MR_int)p.first()).get(),
- materialize(p.second()));
- };
- do {
- if (!skip)
- step++;
- if (!skip && Config.trace_execution) {
- tabs = Interpreter.tabs(Interpreter.tab_count);
- System.out.println(tabs+" Superstep "+step+":");
- System.out.println(tabs+" messages: "+msgs);
- System.out.println(tabs+" state: "+state);
- for ( int i = 0; i < cache.size(); i++)
- if (cache.get(i) instanceof Bag && ((Bag)cache.get(i)).size() > 0)
- System.out.println(tabs+" cache "+i+": "+cache.get(i));
- };
- result = (Tuple)superstep.eval(new Tuple(cache,msgs,state,new MR_string("")));
- Bag new_msgs = (Bag)result.get(0);
- state = result.get(1);
- exit = ((MR_bool)result.get(2)).get();
- skip = new_msgs == SystemFunctions.bsp_empty_bag;
- if ((!skip || exit) && Config.trace_execution)
- System.out.println(tabs+" result: "+result);
- final Iterator<MRData> iter = new_msgs.iterator();
- msgs = new Bag(new BagIterator() {
- public boolean hasNext () {
- return iter.hasNext();
- }
- public MRData next () {
- return ((Tuple)iter.next()).get(1);
- }
- });
- } while (!exit);
- MRData[] data = new MRData[source.length];
- for ( int i = 0; i < data.length; i++ )
- data[i] = getCache(source[i]);
- if (order && data[0] instanceof Bag) {
- final Iterator<MRData> iter = ((Bag)data[0]).iterator();
- return new Bag(new BagIterator() {
- public boolean hasNext () {
- return iter.hasNext();
- }
- public MRData next () {
- return ((Tuple)iter.next()).get(0);
- }
- });
- };
- if (data.length == 1)
- return data[0];
- else return new Tuple(data);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Materialization.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Materialization.gen b/src/main/java/core/Materialization.gen
deleted file mode 100644
index 5683699..0000000
--- a/src/main/java/core/Materialization.gen
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.*;
-
-
-class Domains {
- public Trees domains;
- public Trees repeats;
- public Domains ( Trees d, Trees r ) { domains = d; repeats = r; }
-}
-
-
-/** if the plan refers to a variable bound to a stream-based Bag and occurs in the code
- * multiple times, embed code to materialize this Bag in memory
- */
-final public class Materialization extends Translator {
-
- // is this a direct-access term? (not the results of a bulk operation)
- private static boolean access_variable ( Tree e ) {
- match e {
- case nth(`x,_):
- return access_variable(x);
- case union_value(`x):
- return access_variable(x);
- case index(`x,`n):
- return access_variable(x);
- case `v:
- if (v.is_variable())
- return true;
- };
- return false;
- }
-
- private static Domains new_domain ( Trees vars, Tree e, Domains d ) {
- if (!access_variable(e))
- return materialize(vars,e,d);
- Domains nd = new Domains(d.domains,d.repeats);
- if ((d.domains.member(e) || !free_variables(e,vars).is_empty())
- && !d.repeats.member(e))
- nd.repeats = nd.repeats.cons(e);
- nd.domains = nd.domains.cons(e);
- return nd;
- }
-
- private static Domains union ( Domains xd, Domains yd ) {
- Domains nd = new Domains(xd.domains,xd.repeats);
- for ( Tree y: yd.domains )
- if (!nd.domains.member(y))
- nd.domains = nd.domains.cons(y);
- for ( Tree y: yd.repeats )
- if (!nd.repeats.member(y))
- nd.repeats = nd.repeats.cons(y);
- return nd;
- }
-
- final static int unionM = ClassImporter.find_method_number("plus",#[bag(any),bag(any)]);
-
- private static Domains materialize ( Trees vars, Tree e, Domains d ) {
- match e {
- case lambda(`v,`b):
- return materialize(#[`v],b,d);
- case cmap(lambda(`v,`b),`s):
- return materialize(#[`v],b,new_domain(vars,s,d));
- case map(lambda(`v,`b),`s):
- return materialize(#[`v],b,new_domain(vars,s,d));
- case filter(lambda(`v1,`b1),lambda(`v2,`b2),`s):
- return materialize(#[`v1],b1,materialize(#[`v2],b2,new_domain(vars,s,d)));
- case aggregate(lambda(`v,`b),`z,`s):
- return materialize(#[`v],b,new_domain(vars,s,d));
- case groupBy(`s):
- return new_domain(vars,s,d);
- case orderBy(`s):
- return new_domain(vars,s,d);
- case mapReduce2(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`v,`b),`x,`y,`o):
- return materialize(#[`v],b,materialize(#[`v1],b1,materialize(#[`v2],b2,
- new_domain(vars,x,new_domain(vars,y,d)))));
- case join(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`v,`b),`x,`y):
- return materialize(#[`v],b,materialize(#[`v1],b1,materialize(#[`v2],b2,
- new_domain(vars,x,new_domain(vars,y,d)))));
- case crossProduct(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`v,`b),`x,`y):
- return materialize(#[`v],b,materialize(#[`v1],b1,materialize(#[`v2],b2,
- new_domain(vars,x,new_domain(vars,y,d)))));
- case let(`v,`x,`y):
- Domains nd = materialize(vars.cons(v),y,materialize(vars,x,d));
- Trees zs = #[];
- for ( Tree z: nd.repeats )
- if (!v.equals(z))
- zs = zs.cons(z);
- nd.repeats = zs;
- return nd;
- case if(`p,`x,`y):
- Domains nd = materialize(vars,p,d);
- return union(materialize(vars,x,nd),
- materialize(vars,y,nd));
- case callM(union,_,`x,`y):
- return new_domain(vars,x,new_domain(vars,y,d));
- case callM(_,`k,`x,`y):
- if (((LongLeaf)k).value() != unionM)
- fail;
- return new_domain(vars,x,new_domain(vars,y,d));
- case `f(...as):
- Domains nd = new Domains(d.domains,d.repeats);
- for ( Tree a: as )
- nd = materialize(vars,a,nd);
- return nd;
- };
- return d;
- }
-
- public static Tree materialize_terms ( Tree e ) {
- Domains d = materialize(#[],e,new Domains(#[],#[]));
- for ( Tree x: d.repeats )
- e = subst(x,#<materialize(`x)>,e);
- return e;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/MethodInfo.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/MethodInfo.java b/src/main/java/core/MethodInfo.java
deleted file mode 100644
index 2dc9ad1..0000000
--- a/src/main/java/core/MethodInfo.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.lang.reflect.Method;
-import org.apache.mrql.gen.*;
-
-
-/** class for storing Java method information */
-final public class MethodInfo implements Comparable<MethodInfo> {
- public String name;
- public Trees signature;
- public Method method;
-
- MethodInfo ( String n, Trees s, Method m ) {
- name = n;
- signature = s;
- method = m;
- }
-
- public int compareTo ( MethodInfo x ) {
- int c = name.compareTo(x.name);
- if (c != 0)
- return c;
- if (signature.length() < x.signature.length())
- return -1;
- if (signature.length() > x.signature.length())
- return 1;
- // handles overloading: more specific method signatures first
- for ( int i = 1; i < signature.length(); i++ ) {
- int ct = TypeInference.compare_types(signature.nth(i),x.signature.nth(i));
- if (ct != 0)
- return ct;
- };
- return TypeInference.compare_types(signature.nth(0),x.signature.nth(0));
- }
-
- public boolean equals ( Object x ) {
- return name.equals(((MethodInfo)x).name)
- && signature.equals(((MethodInfo)x).signature);
- }
-}
[07/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Normalization.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Normalization.gen b/src/main/java/core/Normalization.gen
deleted file mode 100644
index 37d628e..0000000
--- a/src/main/java/core/Normalization.gen
+++ /dev/null
@@ -1,406 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-
-
-/** normalize algebraic expressions to more efficient forms using heuristic rules */
-public class Normalization extends Translator {
-
- /** given that pattern=e, find the bindings of the pattern variables */
- static Trees bind_pattern ( Tree pattern, Tree e ) {
- Trees args = #[];
- match pattern {
- case tuple(...pl):
- int i = 0;
- for ( Tree p: pl ) {
- args = args.append(bind_pattern(p,#<nth(`e,`i)>));
- i++;
- }
- case record(...bl):
- Trees attrs = #[];
- for ( Tree b: bl )
- match b {
- case bind(`n,`p):
- args = args.append(bind_pattern(p,#<project(`e,`n)>));
- if (attrs.member(n))
- error("Duplicate record attribute name: "+n);
- attrs = attrs.append(n);
- };
- case typed(`p,`t):
- args = bind_pattern(p,#<typed(`e,`t)>);
- case list(...pl):
- int i = 0;
- for ( Tree p: pl ) {
- args = args.append(bind_pattern(p,#<index(`e,`i)>));
- i++;
- };
- args = args.append(#<call(eq,call(count,`e),`i)>);
- case call(`c,...s):
- Tree ci = data_constructors.lookup(c.toString());
- if (ci == null)
- error("Undefined data constructor: "+c);
- match ci {
- case `dname(`n,`tp):
- args = args.append(#<call(eq,union_tag(`e),`n)>);
- args = args.append(bind_pattern(s.length() == 1 ? s.head() : #<tuple(...s)>,
- #<typed(union_value(`e),`tp)>));
- };
- case any: ;
- case `v:
- if (!v.is_variable()) // constant in pattern
- args = #[call(eq,`e,`v)];
- else if (st.lookup(v.toString()) != null // repeated pattern variable
- && !(e.is_variable() && st.lookup(v.toString()).is_variable())) // exception
- args = #[call(eq,`e,`(st.lookup(v.toString())))];
- else st.insert(v.toString(),e); // new pattern variable
- };
- return args;
- }
-
- private static Tree make_tuple ( Trees pl ) {
- if (pl.length() == 1)
- return pl.head();
- return #<tuple(...pl)>;
- }
-
- /** remove group-bys and order-bys from the MRQL queries */
- static Tree remove_groupby ( Tree e ) {
- Tree ret = #<error>;
- match e {
- case select(distinct,`u,from(...bl),where(`c),groupby(...gl),orderby(...ol)):
- ret = #<select(none,tuple(`u,`u),from(...bl),where(`c),groupby(...gl),orderby(...ol))>;
- ret = #<cmap(lambda(tuple(key,group),list(key)),groupBy(`ret))>;
- return remove_groupby(ret);
- case select(none,`u,from(...bl),where(`c),groupby(),orderby()):
- return remove_groupby(#<select(`u,from(...bl),where(`c))>);
- case select(none,`u,from(...bl),where(`c),groupby(...gl),orderby(`l,...ol)):
- Tree tol = make_tuple(ol);
- ret = #<cmap(lambda(tuple(key,group),group),
- orderBy(select(none,tuple(`tol,`u),
- from(...bl),
- where(`c),groupby(...gl),orderby())))>;
- return (l.equals(#<none>))
- ? remove_groupby(ret)
- : #<range(`(remove_groupby(ret)),0,`l)>;
- case select(none,`u,from(...bl),where(`c),groupby(`h,...gl),orderby()):
- Trees pl = #[];
- Trees ul = #[];
- Trees ql = #[];
- for ( Tree b: bl )
- match b {
- case bind(`p,`d):
- pl = pl.append(p);
- };
- Trees pvs = #[];
- for ( Tree g: gl )
- match g {
- case bind(`p,`d):
- ql = ql.append(p);
- ul = ul.append(d);
- pvs = pvs.append(pattern_variables(p));
- };
- Tree tql = make_tuple(ql);
- Tree tul = make_tuple(ul);
- Tree tpl = make_tuple(pl);
- Trees xl = #[];
- Trees partl = #[];
- for ( Tree x: pattern_variables(#<tuple(...pl)>) )
- if (!pvs.member(x)) {
- partl = partl.append(#<bind(`x,`x)>);
- match rename(#<select(`x,from(bind(`tpl,group)),where(true))>) {
- case select(`hd,`binds,...):
- xl = xl.append(#<bind(`x,bag(select(`hd,`binds,where(true))))>);
- }
- };
- match rename(#<select(record(...partl),from(bind(`tpl,group)),where(true))>) {
- case select(`hd,`binds,...):
- xl = xl.cons(#<bind(partition,bag(select(`hd,`binds,where(true))))>);
- }
- tpl = subst(#<any>,#<0>,tpl);
- ret = #<select(`u,from(bind(tuple(`tql,group),
- groupBy(select(tuple(`tul,`tpl),from(...bl),where(`c)))),
- ...xl),where(`h))>;
- return remove_groupby(ret);
- case intersect(`x,`y):
- return remove_groupby(#<select(x,from(bind(x,`x),bind(y,`y)),
- where(call(eq,x,y)))>);
- case except(`x,`y):
- return remove_groupby(#<select(x,from(bind(x,`x)),
- where(call(not,call(exists,select(y,from(bind(y,`y)),
- where(call(eq,x,y)))))))>);
- case member(`x,`y):
- return remove_groupby(#<call(exists,select(y,from(bind(y,`y)),
- where(call(eq,y,`x))))>);
- case call(gen,`min,`max,`size):
- return #<gen(`(remove_groupby(min)),`(remove_groupby(max)),`(remove_groupby(size)))>;
- case call(avg,`s):
- return remove_groupby(#<call(avg_value,call(avg_aggr,`s))>);
- case call(`f,...al):
- Tree macro = global_macros.lookup(f.toString());
- if (macro == null)
- fail;
- match macro {
- case macro(params(...pl),`body):
- Tree b = rename(remove_groupby(body));
- if (pl.length() != al.length())
- fail;
- for ( ; !pl.is_empty(); pl = pl.tail(), al = al.tail() )
- b = subst(pl.head(),remove_groupby(al.head()),b);
- return b;
- }
- case call(`f,...al):
- if (#[cmap,join,mapReduce,mapReduce2,groupBy,orderBy,tuple,bag,list,set].member(f))
- return remove_groupby(#<`(f.toString())(...al)>);
- else fail
- case project(`x,`a):
- return #<project(`(remove_groupby(x)),`a)>;
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(remove_groupby(a));
- return #<`f(...bl)>;
- case `v:
- if (v.is_variable()) {
- ret = global_vars.lookup(v.toString());
- if (ret == null)
- return v;
- else if (!v.equals(ret))
- return remove_groupby(ret);
- }
- };
- return e;
- }
-
- private static Tree make_and ( Trees tests ) {
- if (tests.is_empty())
- return #<true>;
- Tree e = tests.head();
- for ( Tree t: tests.tail() )
- e = #<call(and,`e,`t)>;
- return e;
- }
-
- private static Trees rename_list ( Trees al ) {
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(rename(a));
- return bl;
- }
-
- /** compile away patterns and rename local variables of an MRQL expression e with unique names */
- static Tree rename ( Tree e ) {
- Tree ret = #<error>;
- match e {
- case `v:
- if (!v.is_variable())
- fail;
- ret = st.lookup(v.toString());
- if (ret==null)
- return v;
- else return ret;
- case select(`u,from(...bl),where(`c)):
- st.begin_scope();
- Trees binds = #[];
- Trees tests = #[];
- for ( Tree b: bl )
- match b {
- case bind(`p,`d):
- Tree x = new_var();
- binds = binds.append(#<bind(`x,`(rename(d)))>);
- tests = tests.append(bind_pattern(p,x));
- };
- c = make_and(tests.cons(c));
- ret = #<select(`(rename(u)),
- from(...binds),
- where(`(rename(c))))>;
- st.end_scope();
- return ret;
- case lambda(`p,`b):
- st.begin_scope();
- Tree nv = new_var();
- if (!bind_pattern(p,nv).is_empty())
- error("Lambda patterns must be irrefutable: "+print_query(e));
- ret = #<lambda(`nv,`(rename(b)))>;
- st.end_scope();
- return ret;
- case function(tuple(...params),`outp,`body):
- st.begin_scope();
- Trees ps = #[];
- Trees vs = #[];
- for ( Tree p: params )
- match p {
- case `bind(`v,`tp):
- Tree nv = new_var();
- if (vs.member(v))
- error("Duplicate function parameters: "+print_query(e));
- vs = vs.append(v);
- ps = ps.append(#<`bind(`nv,`tp)>);
- st.insert(v.toString(),nv);
- };
- ret = #<function(tuple(...ps),`outp,`(rename(body)))>;
- st.end_scope();
- return ret;
- case let(`p,`u,`b):
- Tree ne = rename(u);
- st.begin_scope();
- Tree nv = new_var();
- if (!bind_pattern(p,nv).is_empty())
- error("Let patterns must be irrefutable: "+print_query(e));
- ret = #<let(`nv,`ne,`(rename(b)))>;
- st.end_scope();
- return ret;
- case case(`u,...cs):
- Trees rs = cs.reverse();
- Tree nu = rename(u);
- match rs.head() {
- case case(`p,`b):
- Trees conds = bind_pattern(p,nu);
- if (!conds.is_empty())
- error("Non-exhaustive case "+print_query(p)+" in "+print_query(e));
- ret = b;
- };
- for ( Tree c: rs.tail() )
- match c {
- case case(`p,`b):
- Trees conds = bind_pattern(p,nu);
- if (!conds.is_empty())
- ret = #<if(`(make_and(conds)),`b,`ret)>;
- else error("Unreachable case "+print_query(p)+" in "+print_query(e));
- };
- return rename(ret);
- case project(`u,`a):
- return #<project(`(rename(u)),`a)>;
- case bind(`a,`u):
- return #<bind(`a,`(rename(u)))>;
- case loop(lambda(tuple(...vs),`b),`s,`n):
- return #<loop(lambda(tuple(...vs),`(rename(b))),`(rename(s)),`n)>;
- case `f(...al):
- Trees bl = rename_list(al);
- return #<`f(...bl)>;
- };
- return e;
- }
-
- private static Trees has_existential ( Tree e ) {
- match e {
- case call(and(`x,`y)):
- Trees xs = has_existential(x);
- Trees ys = has_existential(y);
- return #[call(and(`(xs.head()),`(ys.head())),...(xs.tail()),...(ys.tail()))];
- case call(exists,select(...)):
- return #[true,`e];
- case call(not,call(all,select(...l))):
- return #[true,call(exists,select(...l))];
- };
- return #[`e];
- }
-
- /** normalize algebraic expressions to more efficient forms using heuristic rules */
- public static Tree normalize ( Tree e ) {
- match e {
- case select(`u,from(),where(true)):
- return normalize(#<bag(`u)>);
- case select(`u,from(),where(`p)):
- return normalize(#<if(`p,bag(`u),bag())>);
- case select(`u,from(bind(`v,`d)),where(true)):
- if (u.equals(v))
- return normalize(d);
- else fail
- case select(`u,from(...bl,bind(`v,select(`iu,from(...ibl),where(`ic))),...al),where(`c)):
- return normalize(#<select(`u,from(...bl,...ibl,bind(`v,bag(`iu)),...al),
- where(call(and,`c,`ic)))>);
- case select(`u,from(...bl,bind(`v,bag(`d)),...al),`c):
- if (!is_pure(d) && occurences(v,#<f(`c,`u,...al)>) > 1) // duplicated side-effects
- fail;
- return normalize(#<select(`(subst(v,d,u)),
- from(...bl,...(subst_list(v,d,al))),
- `(subst(v,d,c)))>);
- case select(`u,from(...bl),where(`c)):
- Trees es = has_existential(c);
- if (es.length() <= 1)
- fail;
- Trees binds = bl;
- Trees preds = #[`(es.head())];
- for ( Tree x: es.tail() )
- match x {
- case call(exists,select(`p,from(...bl2),where(`c2))):
- preds = preds.cons(p).cons(c2);
- binds = binds.append(bl2);
- };
- return normalize(#<select(`u,from(...binds),where(`(make_and(preds))))>);
- case let_bind(`v,`x,`y):
- return #<let(`v,`(normalize(x)),`(normalize(y)))>;
- case call(eq,tuple(...l),`x):
- Tree pl = #<true>;
- int i = 0;
- for ( Tree y: l ) {
- pl = #<call(and,`pl,call(eq,`y,nth(`x,`i)))>;
- i++;
- };
- return normalize(pl);
- case call(eq,`x,tuple(...l)):
- Tree pl = #<true>;
- int i = 0;
- for (Tree y: l) {
- pl = #<call(and,`pl,call(eq,nth(`x,`i),`y))>;
- i++;
- };
- return normalize(pl);
- case call(and,true,`u): return normalize(u);
- case call(and,`u,true): return normalize(u);
- case call(and,false,`u): return #<false>;
- case call(and,`u,false): return #<false>;
- case call(or,true,`u): return #<true>;
- case call(or,`u,true): return #<true>;
- case call(or,false,`u): return normalize(u);
- case call(or,`u,false): return normalize(u);
- case call(not,true): return #<false>;
- case call(not,false): return #<true>;
- case if(true,`e1,`e2): return normalize(e1);
- case if(false,`e1,`e2): return normalize(e2);
- case nth(tuple(...al),`n):
- if (!n.is_long())
- fail;
- int i = (int)n.longValue();
- if ( i >= 0 && i < al.length() )
- return normalize(al.nth(i));
- case project(record(...bl),`a):
- for ( Tree b: bl )
- match b {
- case bind(`v,`u): if (v.equals(a)) return normalize(u);
- };
- error("Wrong projection: "+print_query(e));
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(normalize(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- /** normalize algebraic expressions to more efficient forms using heuristic rules */
- public static Tree normalize_all ( Tree e ) {
- Tree ne = normalize(e);
- if (e.equals(ne))
- return e;
- else return normalize(ne);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/ParsedDataSource.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/ParsedDataSource.java b/src/main/java/core/ParsedDataSource.java
deleted file mode 100644
index e000405..0000000
--- a/src/main/java/core/ParsedDataSource.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** A data source for a text HDFS file along with the parser to parse it */
-public class ParsedDataSource extends DataSource {
- public Class<? extends Parser> parser;
- public Trees args;
-
- ParsedDataSource ( int source_num,
- String path,
- Class<? extends Parser> parser,
- Trees args,
- Configuration conf ) {
- super(source_num,path,Evaluator.evaluator.parsedInputFormat(),conf);
- this.parser = parser;
- this.args = args;
- }
-
- ParsedDataSource ( String path,
- Class<? extends Parser> parser,
- Trees args,
- Configuration conf ) {
- super(-1,path,Evaluator.evaluator.parsedInputFormat(),conf);
- this.parser = parser;
- this.args = args;
- }
-
- public String toString () {
- try {
- String pn = "";
- for ( String k: DataSource.parserDirectory.keySet() )
- if (DataSource.parserDirectory.get(k).equals(parser))
- pn = k;
- return "Text"+separator+source_num+separator+pn+separator+path
- +separator+(new Node("args",args)).toString();
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Parser.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Parser.java b/src/main/java/core/Parser.java
deleted file mode 100644
index c1728f1..0000000
--- a/src/main/java/core/Parser.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import org.apache.hadoop.fs.FSDataInputStream;
-
-
-/** for a new text data source, you must implement a new parser */
-interface Parser {
- public void initialize ( Trees args );
- public Tree type ();
- public void open ( String file );
- public void open ( FSDataInputStream fsin, long start, long end );
- public String slice ();
- public Bag parse ( String s );
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Plan.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Plan.java b/src/main/java/core/Plan.java
deleted file mode 100644
index c4951bf..0000000
--- a/src/main/java/core/Plan.java
+++ /dev/null
@@ -1,362 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import java.util.Random;
-import java.util.ArrayList;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.SequenceFile.Sorter;
-
-
-/** A physical plan (a superclass for both MapReduce, BSP, and Spark plans) */
-public class Plan {
- public static Configuration conf;
- static ArrayList<String> temporary_paths = new ArrayList<String>();
- private static Random random_generator = new Random();
- final static int max_input_files = 100;
-
- /** generate a new path name in HDFS to store intermediate results */
- public static String new_path ( Configuration conf ) throws IOException {
- String dir = (Config.local_mode)
- ? ((Config.tmpDirectory == null) ? "/tmp/mrql" : Config.tmpDirectory)
- : "mrql";
- Path p;
- do {
- p = new Path(dir+"/mrql"+random_generator.nextInt(1000000));
- } while (p.getFileSystem(conf).exists(p));
- String path = p.toString();
- temporary_paths.add(path);
- DataSource.dataSourceDirectory.distribute(conf);
- return path;
- }
-
- /** remove all temporary files */
- public static void clean () throws IOException {
- for (String p: temporary_paths)
- try {
- Path path = new Path(p);
- path.getFileSystem(conf).delete(path,true);
- } catch (Exception ex) {
- FileSystem.getLocal(conf).delete(new Path(p),true);
- };
- temporary_paths.clear();
- DataSource.dataSourceDirectory.clear();
- }
-
- /** return the data set size in bytes */
- public final static long size ( DataSet s ) {
- return s.size(conf);
- }
-
- /** the cache that holds all local data in memory */
- static Tuple cache;
-
- /** return the cache element at location loc */
- public static synchronized MRData getCache ( int loc ) {
- return cache.get(loc);
- }
-
- /** set the cache element at location loc to value and return ret */
- public static synchronized MRData setCache ( int loc, MRData value, MRData ret ) {
- if (value instanceof Bag)
- ((Bag)value).materialize();
- cache.set(loc,value);
- return ret;
- }
-
- /** put the jar file that contains the compiled MR functional parameters into the TaskTracker classpath */
- final static void distribute_compiled_arguments ( Configuration conf ) {
- try {
- if (!Config.compile_functional_arguments)
- return;
- Path local_path = new Path("file://"+Compiler.jar_path);
- if (Config.spark_mode)
- conf.set("mrql.jar.path",local_path.toString());
- else {
- // distribute the jar file with the compiled arguments to all clients
- Path hdfs_path = new Path("mrql-tmp/class"+random_generator.nextInt(1000000)+".jar");
- FileSystem fs = hdfs_path.getFileSystem(conf);
- fs.copyFromLocalFile(false,true,local_path,hdfs_path);
- temporary_paths.add(hdfs_path.toString());
- conf.set("mrql.jar.path",hdfs_path.toString());
- }
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
-
- /** retrieve the compiled functional argument of code */
- final static Function functional_argument ( Configuration conf, Tree code ) {
- Node n = (Node)code;
- if (n.name().equals("compiled"))
- try {
- // if the clent has not received the jar file with the compiled arguments, copy the file from HDFS
- if (Compiler.jar_path == null) {
- Path hdfs_path = new Path(conf.get("mrql.jar.path"));
- String local_path = Compiler.tmp_dir+"/mrql_args_"+random_generator.nextInt(1000000)+".jar";
- FileSystem fs = hdfs_path.getFileSystem(conf);
- fs.copyToLocalFile(false,hdfs_path,new Path("file://"+local_path));
- Compiler.jar_path = local_path;
- };
- return Compiler.compiled(conf.getClassLoader(),n.children().nth(0).toString());
- } catch (Exception ex) {
- System.err.println("*** Warning: Unable to retrieve the compiled lambda: "+code);
- return ((Lambda) Interpreter.evalE(n.children().nth(1))).lambda();
- }
- else if (code.equals(Interpreter.identity_mapper))
- return new Function () {
- public MRData eval ( final MRData x ) { return new Bag(x); }
- };
- else return ((Lambda) Interpreter.evalE(code)).lambda();
- }
-
- /** comparator for MRData keys */
- public final static class MRContainerKeyComparator implements RawComparator<MRContainer> {
- int[] container_size;
-
- public MRContainerKeyComparator () {
- container_size = new int[1];
- }
-
- final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
- return MRContainer.compare(x,xs,xl,y,ys,yl,container_size);
- }
-
- final public int compare ( MRContainer x, MRContainer y ) {
- return x.compareTo(y);
- }
- }
-
- /** The source physical operator for binary files */
- public final static DataSet binarySource ( int source_num, String file ) {
- return new DataSet(new BinaryDataSource(source_num,file,conf),0,0);
- }
-
- /** The source physical operator for binary files */
- public final static DataSet binarySource ( String file ) {
- return new DataSet(new BinaryDataSource(-1,file,conf),0,0);
- }
-
- /** splits the range min..max into multiple ranges, one for each mapper */
- public final static DataSet generator ( int source_num, long min, long max, long split_length ) throws Exception {
- if (min > max)
- throw new Error("Wrong range: "+min+"..."+max);
- if (split_length < 1)
- if (Config.bsp_mode)
- split_length = (max-min)/Config.nodes+1;
- else split_length = Config.range_split_size;
- DataSet ds = new DataSet(0,0);
- long i = min;
- while (i+split_length <= max) {
- String file = new_path(conf);
- Path path = new Path(file);
- SequenceFile.Writer writer
- = SequenceFile.createWriter(path.getFileSystem(conf),conf,path,
- MRContainer.class,MRContainer.class,
- SequenceFile.CompressionType.NONE);
- writer.append(new MRContainer(new MR_long(i)),
- new MRContainer(new Tuple(new MR_long(i),new MR_long(split_length))));
- writer.close();
- ds.source.add(new GeneratorDataSource(source_num,file,conf));
- i += split_length;
- };
- if (i <= max) {
- String file = new_path(conf);
- Path path = new Path(file);
- SequenceFile.Writer writer
- = SequenceFile.createWriter(path.getFileSystem(conf),conf,path,
- MRContainer.class,MRContainer.class,
- SequenceFile.CompressionType.NONE);
- writer.append(new MRContainer(new MR_long(i)),
- new MRContainer(new Tuple(new MR_long(i),new MR_long(max-i+1))));
- writer.close();
- ds.source.add(new GeneratorDataSource(source_num,file,conf));
- };
- return ds;
- }
-
- /** splits the range min..max into multiple ranges, one for each mapper */
- public final static DataSet generator ( long min, long max, long split_length ) throws Exception {
- return generator(-1,min,max,split_length);
- }
-
- /** The source physical operator for parsing text files */
- public final static DataSet parsedSource ( int source_num, Class<? extends Parser> parser, String file, Trees args ) {
- return new DataSet(new ParsedDataSource(source_num,file,parser,args,conf),0,0);
- }
-
- /** The source physical operator for parsing text files */
- public final static DataSet parsedSource ( Class<? extends Parser> parser, String file, Trees args ) {
- return new DataSet(new ParsedDataSource(file,parser,args,conf),0,0);
- }
-
- /** merge the sorted files of the data source */
- public final static Bag merge ( final DataSource s ) throws Exception {
- Path path = new Path(s.path);
- final FileSystem fs = path.getFileSystem(conf);
- final FileStatus[] ds
- = fs.listStatus(path,
- new PathFilter () {
- public boolean accept ( Path path ) {
- return !path.getName().startsWith("_");
- }
- });
- int dl = ds.length;
- if (dl == 0)
- return new Bag();
- Path[] paths = new Path[dl];
- for ( int i = 0; i < dl; i++ )
- paths[i] = ds[i].getPath();
- if (dl > Config.max_merged_streams) {
- if (Config.trace)
- System.err.println("*** Merging "+dl+" files");
- Path out_path = new Path(new_path(conf));
- SequenceFile.Sorter sorter
- = new SequenceFile.Sorter(fs,new MRContainerKeyComparator(),
- MRContainer.class,MRContainer.class,conf);
- sorter.merge(paths,out_path);
- paths = new Path[1];
- paths[0] = out_path;
- };
- final int n = paths.length;
- SequenceFile.Reader[] sreaders = new SequenceFile.Reader[n];
- for ( int i = 0; i < n; i++ )
- sreaders[i] = new SequenceFile.Reader(fs,paths[i],conf);
- final SequenceFile.Reader[] readers = sreaders;
- final MRContainer[] keys_ = new MRContainer[n];
- final MRContainer[] values_ = new MRContainer[n];
- for ( int i = 0; i < n; i++ ) {
- keys_[i] = new MRContainer();
- values_[i] = new MRContainer();
- };
- return new Bag(new BagIterator () {
- int min = 0;
- boolean first = true;
- final MRContainer[] keys = keys_;
- final MRContainer[] values = values_;
- final MRContainer key = new MRContainer();
- final MRContainer value = new MRContainer();
- public boolean hasNext () {
- if (first)
- try {
- first = false;
- for ( int i = 0; i < n; i++ )
- if (readers[i].next(key,value)) {
- keys[i].set(key.data());
- values[i].set(value.data());
- } else {
- keys[i] = null;
- readers[i].close();
- }
- } catch (IOException e) {
- throw new Error("Cannot merge values from an intermediate result");
- };
- min = -1;
- for ( int i = 0; i < n; i++ )
- if (keys[i] != null && min < 0)
- min = i;
- else if (keys[i] != null && keys[i].compareTo(keys[min]) < 0)
- min = i;
- return min >= 0;
- }
- public MRData next () {
- try {
- MRData res = values[min].data();
- if (readers[min].next(key,value)) {
- keys[min].set(key.data());
- values[min].set(value.data());
- } else {
- keys[min] = null;
- readers[min].close();
- };
- return res;
- } catch (IOException e) {
- throw new Error("Cannot merge values from an intermediate result");
- }
- }
- });
- }
-
- /** The collect physical operator */
- public final static Bag collect ( final DataSet x, boolean strip ) throws Exception {
- return Evaluator.evaluator.parsedInputFormat().newInstance().collect(x,strip);
- }
-
- /** The collect physical operator */
- public final static Bag collect ( final DataSet x ) throws Exception {
- return collect(x,true);
- }
-
- /** the DataSet union physical operator */
- public final static DataSet merge ( final DataSet x, final DataSet y ) throws IOException {
- DataSet res = x;
- res.source.addAll(y.source);
- return res;
- }
-
- final static MR_long counter_key = new MR_long(0);
- final static MRContainer counter_container = new MRContainer(counter_key);
- final static MRContainer value_container = new MRContainer(new MR_int(0));
-
- /** The cache operator that dumps a bag into an HDFS file */
- public final static DataSet fileCache ( Bag s ) throws IOException {
- String newpath = new_path(conf);
- Path path = new Path(newpath);
- FileSystem fs = path.getFileSystem(conf);
- SequenceFile.Writer writer
- = new SequenceFile.Writer(fs,conf,path,
- MRContainer.class,MRContainer.class);
- long i = 0;
- for ( MRData e: s ) {
- counter_key.set(i++);
- value_container.set(e);
- writer.append(counter_container,value_container);
- };
- writer.close();
- return new DataSet(new BinaryDataSource(0,newpath,conf),0,0);
- }
-
- /** for dumped data to a file, return the MRQL type of the data */
- public final static Tree get_type ( String file ) {
- try {
- Path path = new Path(file);
- FileSystem fs = path.getFileSystem(conf);
- BufferedReader ftp = new BufferedReader(new InputStreamReader(fs.open(path.suffix(".type"))));
- String s[] = ftp.readLine().split("@");
- ftp.close();
- if (s.length != 2 )
- return null;
- if (!s[0].equals("2"))
- throw new Error("The binary file has been created in java mode and cannot be read in hadoop mode");
- return Tree.parse(s[1]);
- } catch (Exception e) {
- return null;
- }
- }
-
- /** create a new PrintStream from the file */
- final static PrintStream print_stream ( String file ) throws Exception {
- Path path = new Path(file);
- FileSystem fs = path.getFileSystem(conf);
- return new PrintStream(fs.create(path));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/PlanGeneration.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/PlanGeneration.gen b/src/main/java/core/PlanGeneration.gen
deleted file mode 100644
index 59b8cb8..0000000
--- a/src/main/java/core/PlanGeneration.gen
+++ /dev/null
@@ -1,716 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.*;
-import java.io.*;
-
-
-/** Generate a physical plan from an algebraic expression */
-final public class PlanGeneration extends AlgebraicOptimization {
-
- /** extract the combiner from the reducer in a MapReduce plan */
- static class Aggregates {
- public static Trees maps = #[];
- public static Trees combines = #[];
- public static Trees reduces = #[];
- public static boolean can_use_combiner = true;
- private static SymbolTable st = new SymbolTable();
-
- private static void clear () {
- maps = #[];
- combines = #[];
- reduces = #[];
- can_use_combiner = true;
- }
-
- /** extends the combines, maps, and reduces lists with new entries */
- private static int union_aggegates ( Tree reduce, Tree map, Tree combine ) {
- Tree m = simplify_all(map);
- Tree c = simplify_all(combine);
- Tree rd = simplify_all(reduce);
- int i = 0;
- for ( Trees r = reduces; !r.is_empty(); r = r.tail(), i++ )
- if (alpha_equivalent(rd,r.head()))
- return i;
- maps = maps.append(m);
- reduces = reduces.append(rd);
- combines = combines.append(subst(#<-1>,#<`i>,c));
- return i;
- }
-
- /** Generate the MR combiner from the MR reducer.
- * Find the aggregation calls (eg, call(avg,_)) in the reducer
- * @param e the body of the reducer
- * @param map the map function
- * @param mvar the variable of the map function
- * @param rvar the variable of the reducer function
- * @return the combiner, but also sets the combines, maps, and reduces lists
- */
- private static Tree derive_combiner ( Tree e, Tree map, Tree mvar, Tree rvar ) {
- Tree gvar = #<nth(`rvar,1)>;
- match e {
- case call(`f,`u):
- for ( Tree monoid: monoids ) // system & user-defined aggregations
- match monoid {
- case `nm(`mtp,`plus,`zero,`unit):
- if (nm.equals(f.toString())) {
- match TypeInference.type_inference2(u) {
- case `S(`tp):
- if (TypeInference.unify(tp,mtp) == null)
- continue;
- case _: throw new Error("Unexpected aggregation: "+e);
- };
- match u {
- case cmap(`m,`v):
- if (!v.equals(gvar) || occurences(rvar,m) > 0)
- fail;
- Tree ev = new_var();
- Tree nv = new_var();
- Tree mv = new_var();
- int i = union_aggegates(e,
- #<aggregate(lambda(`ev,apply(`plus,tuple(nth(`ev,0),apply(`unit,nth(`ev,1))))),
- `zero,cmap(`m,cmap(lambda(x,bag(nth(x,1))),apply(`map,`mvar))))>,
- #<aggregate(lambda(`nv,apply(`plus,tuple(nth(`nv,0),
- nth(nth(`nv,1),-1)))),
- `zero,`gvar)>);
- return simplify_all(#<aggregate(lambda(`mv,apply(`plus,tuple(nth(`mv,0),
- nth(nth(`mv,1),`i)))),
- `zero,`gvar)>);
- case `v:
- if (!v.equals(gvar))
- fail;
- Tree ev = new_var();
- Tree nv = new_var();
- Tree mv = new_var();
- int i = union_aggegates(e,
- #<aggregate(lambda(`ev,apply(`plus,tuple(nth(`ev,0),apply(`unit,nth(`ev,1))))),
- `zero,cmap(lambda(x,bag(nth(x,1))),apply(`map,`mvar)))>,
- #<aggregate(lambda(`nv,apply(`plus,tuple(nth(`nv,0),nth(nth(`nv,1),-1)))),
- `zero,`gvar)>);
- return simplify_all(#<aggregate(lambda(`mv,apply(`plus,tuple(nth(`mv,0),
- nth(nth(`mv,1),`i)))),
- `zero,`gvar)>);
- }
- }
- };
- fail
- case nth(`v,0):
- if (v.is_variable())
- return e;
- else fail
- case `f(...al):
- Trees rs = #[];
- for ( Tree a: al )
- rs = rs.append(derive_combiner(a,map,mvar,rvar));
- return #<`f(...rs)>;
- };
- if (#<nth(`e,1)>.equals(gvar))
- Aggregates.can_use_combiner = false;
- return e;
- }
- }
-
- /** how many times e accesses the bag x? if it's more than one, it can't be streamed */
- private static int number_of_accesses ( Tree x, Tree e ) {
- if (e.equals(x))
- return 1;
- match e {
- case cmap(`m,`s):
- return number_of_accesses(x,m)*10+number_of_accesses(x,s);
- case map(`m,`s):
- return number_of_accesses(x,m)*10+number_of_accesses(x,s);
- case filter(`p,`m,`s):
- return number_of_accesses(x,p)*10+number_of_accesses(x,m)*10+number_of_accesses(x,s);
- case `f(...r):
- int i = 0;
- for ( Tree z: r )
- i += number_of_accesses(x,z);
- return i;
- };
- return 0;
- }
-
- /** can we process the second arg of the MapReducer reducer (a bag) as a stream? */
- public static boolean streamed_MapReduce_reducer ( Tree x ) {
- match x {
- case lambda(`v,`b):
- return number_of_accesses(#<nth(`v,1)>,b) <= 1;
- case compiled(_,lambda(`v,`b)):
- return number_of_accesses(#<nth(`v,1)>,b) <= 1;
- };
- return false;
- }
-
- /** can we process the first arg of the MapReducer2 reducer (a bag) as a stream? */
- public static boolean streamed_MapReduce2_reducer ( Tree x ) {
- match x {
- case lambda(`v,`b):
- return number_of_accesses(#<nth(`v,0)>,b) <= 1;
- case compiled(_,lambda(`v,`b)):
- return number_of_accesses(#<nth(`v,0)>,b) <= 1;
- };
- return false;
- }
-
- /** true if e returns a dataset stored in HDFS */
- public static boolean is_dataset_expr ( Tree e ) {
- match TypeInference.type_inference2(e) {
- case `T(_):
- if (is_persistent_collection(T))
- return true;
- };
- return false;
- }
-
- /** compile an algebraic form to a algebraic plan
- * @param e the algebraic form
- * @return the algebraic plan
- */
- public static Tree makePlan ( Tree e ) {
- match e {
- // combine groupBy with Join (experimental)
- case mapReduce(lambda(`vm,`bm),lambda(`vr,`br),`s,`o):
- if (!Config.use_combiner || !is_dataset_expr(s))
- fail;
- Tree splan = makePlan(s);
- match splan {
- case MapReduce2(lambda(`mvx,bag(tuple(`kx,`mx))),
- lambda(`mvy,bag(tuple(`ky,`my))),
- lambda(`v,cmap(lambda(`x,cmap(lambda(`y,bag(tuple(tuple(`gx,`gy),`b))),
- nth(`vx,1))),
- nth(`vy,0))),
- `X,`Y,`o2):
- if (!vx.equals(v) || !vy.equals(v) || !mx.equals(mvx) || !my.equals(mvy))
- fail;
- Tree gxx = gx;
- Tree gyy = gy;
- if (free_variables(gx,#[]).equals(#[`y]) && free_variables(gy,#[]).equals(#[`x])) {
- gxx = gy;
- gyy = gx;
- } else if (!free_variables(gx,#[]).equals(#[`x]) || !free_variables(gy,#[]).equals(#[`y]))
- fail;
- Aggregates.clear();
- Tree nv = new_var();
- match TypeInference.type_inference(bm) {
- case `S(`tp):
- if (!is_collection(S))
- fail;
- type_env.insert(nv.toString(),tp);
- };
- Tree rd = Aggregates.derive_combiner(br,#<lambda(`vm,`bm)>,vm,vr);
- if (Aggregates.reduces.is_empty())
- fail;
- Tree vr2 = new_var();
- Tree m = simplify_all(#<lambda(`vm,cmap(lambda(`nv,bag(tuple(nth(`nv,0),
- tuple(...(Aggregates.maps))))),`bm))>);
- Tree c = subst(vr,vr2,#<bag(tuple(...(Aggregates.combines)))>);
- c = simplify_all(#<lambda(`vr2,`c)>);
- Tree r = simplify_all(#<lambda(`vr,`rd)>);
- Tree mtp = TypeInference.type_inference(#<bag(tuple(...(Aggregates.maps)))>);
- Tree rtp = #<tuple(`(TypeInference.type_inference(#<nth(`vr,0)>)),`mtp)>;
- type_env.insert(vr.toString(),rtp);
- type_env.insert(vr2.toString(),rtp);
- TypeInference.type_inference(m);
- TypeInference.type_inference(c);
- TypeInference.type_inference(r);
- Tree combiner = makePlan(c);
- Tree reducer = makePlan(r);
- Tree nm = simplify_all(subst(x,#<nth(`vx,0)>,
- subst(y,#<nth(`vy,1)>,
- #<cmap(lambda(x,bag(nth(x,1))),
- cmap(`m,bag(tuple(tuple(`gx,`gy),`b))))>)));
- type_env.insert(v.toString(),TypeInference.type_inference(#<tuple(`mvx,`mvy)>));
- nm = makePlan(nm);
- return #<GroupByJoin(lambda(`mvx,`kx),
- lambda(`mvy,`ky),
- lambda(`x,`gxx),
- lambda(`y,`gyy),
- lambda(`v,`nm),
- `combiner,
- `reducer,
- `X,`Y,`o)>;
- };
- fail
- // extract the mapReduce combiner
- case mapReduce(lambda(`vm,`bm),lambda(`vr,`br),`s,`o):
- if (!Config.use_combiner || !is_dataset_expr(s))
- fail;
- Tree splan = makePlan(s);
- Aggregates.clear();
- Tree nv = new_var();
- match TypeInference.type_inference(bm) {
- case `S(`tp):
- if (!is_collection(S))
- fail;
- type_env.insert(nv.toString(),tp);
- };
- Tree rd = Aggregates.derive_combiner(br,#<lambda(`vm,`bm)>,vm,vr);
- if (!Aggregates.can_use_combiner || Aggregates.reduces.is_empty())
- fail;
- Tree vr2 = new_var();
- Tree m = simplify_all(#<lambda(`vm,cmap(lambda(`nv,bag(tuple(nth(`nv,0),
- tuple(...(Aggregates.maps))))),`bm))>);
- Tree c = subst(vr,vr2,#<bag(tuple(...(Aggregates.combines)))>);
- c = simplify_all(#<lambda(`vr2,`c)>);
- Tree r = simplify_all(#<lambda(`vr,`rd)>);
- Tree mtp = TypeInference.type_inference(#<bag(tuple(...(Aggregates.maps)))>);
- Tree rtp = #<tuple(`(TypeInference.type_inference(#<nth(`vr,0)>)),`mtp)>;
- type_env.insert(vr.toString(),rtp);
- type_env.insert(vr2.toString(),rtp);
- TypeInference.type_inference(m);
- TypeInference.type_inference(c);
- TypeInference.type_inference(r);
- Tree combiner = makePlan(c);
- Tree reducer = makePlan(r);
- match splan {
- // if the MapCombineReduce input is a join, push the combiner to the join
- case MapReduce2(`mx,`my,lambda(`rv,`rb),`x,`y,`o2):
- Tree nr = makePlan(simplify_all(#<lambda(`rv,cmap(`m,`rb))>));
- return #<MapReduce(lambda(`vm,bag(`vm)),`reducer,
- MapCombineReduce2(`mx,`my,`combiner,`nr,`x,`y,`o2),`o)>;
- case `input:
- return #<MapCombineReduce(`(makePlan(m)),`combiner,`reducer,`input,`o)>;
- };
- fail
- case mapReduce(`m,`r,`s,`o):
- if (is_dataset_expr(s))
- return #<MapReduce(`(makePlan(m)),
- `(makePlan(r)),
- `(makePlan(s)),`o)>;
- else fail
- case mapReduce2(`mx,`my,`r,`x,`y,`o):
- if (is_dataset_expr(x) && is_dataset_expr(y) && streamed_MapReduce2_reducer(r))
- return #<MapReduce2(`(makePlan(mx)),
- `(makePlan(my)),
- `(makePlan(r)),
- `(makePlan(x)),
- `(makePlan(y)),`o)>;
- else fail
- case mapReduce2(`mx,`my,lambda(`v,`b),`x,`y,`o):
- if (!is_dataset_expr(x) || !is_dataset_expr(y))
- fail;
- // if the join reducer is not streaming, switch the inputs
- Tree nv = new_var();
- Tree nr = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
- subst(#<nth(`v,1)>,#<nth(`nv,0)>,b));
- nr = #<lambda(`nv,`nr)>;
- type_env.insert(nv.toString(),TypeInference.type_inference(#<tuple(nth(`v,1),nth(`v,0))>));
- return #<MapReduce2(`(makePlan(my)),
- `(makePlan(mx)),
- `(makePlan(nr)),
- `(makePlan(y)),
- `(makePlan(x)),`o)>;
- case crossProduct(`mx,`my,`r,`x,`y):
- if (is_dataset_expr(x) && is_dataset_expr(y))
- return #<CrossProduct(`(makePlan(mx)),
- `(makePlan(my)),
- `(makePlan(r)),
- `(makePlan(x)),
- `(makePlan(y)))>;
- else fail
- case cmap(`m,`s):
- if (is_dataset_expr(s))
- return #<cMap(`(makePlan(m)),
- `(makePlan(s)))>;
- else fail
- case cmap(lambda(`v,if(`p,`T(`u),`S())),`s):
- if (false && is_collection(T) && is_collection(S))
- return makePlan(#<filter(lambda(`v,`p),lambda(`v,`u),`s)>);
- else fail
- case call(source,binary,`file,`tp):
- return #<BinarySource(`file,`tp)>;
- case call(source,gen,`f,`len,`ulen):
- return #<SequenceSource(`(makePlan(f)),`(makePlan(len)),
- `(makePlan(ulen)))>;
- case call(source,`parser,`file,...args):
- Trees el = #[];
- for ( Tree a: args )
- el = el.append(makePlan(a));
- return #<ParsedSource(`parser,`(makePlan(file)),...el)>;
- case type(`x): return e;
- case gen(`min,`max,`size):
- return #<Generator(`(makePlan(min)),`(makePlan(max)),`(makePlan(size)))>;
- case repeat(lambda(`v,`b),`s,`n):
- if (!is_dataset_expr(s))
- fail;
- repeat_variables = repeat_variables.cons(v);
- return #<Repeat(lambda(`v,`(makePlan(b))),`(makePlan(s)),
- `(makePlan(n)))>;
- case repeat(lambda(`v,`b),`s):
- if (!is_dataset_expr(s))
- fail;
- repeat_variables = repeat_variables.cons(v);
- return #<Repeat(lambda(`v,`(makePlan(b))),`(makePlan(s)),`(Integer.MAX_VALUE))>;
- case closure(lambda(`v,`b),`s,`n):
- if (!is_dataset_expr(s))
- fail;
- repeat_variables = repeat_variables.cons(v);
- return #<Closure(lambda(`v,`(makePlan(b))),`(makePlan(s)),
- `(makePlan(n)))>;
- case closure(lambda(`v,`b),`s):
- if (!is_dataset_expr(s))
- fail;
- repeat_variables = repeat_variables.cons(v);
- return #<Closure(lambda(`v,`(makePlan(b))),`(makePlan(s)),`(Integer.MAX_VALUE))>;
- case loop(lambda(tuple(...vs),`b),`s,`n):
- repeat_variables = repeat_variables.append(vs);
- return #<Loop(lambda(tuple(...vs),`(makePlan(b))),`(makePlan(s)),`(makePlan(n)))>;
- case record(...bl):
- Trees el = #[];
- for ( Tree b: bl )
- match b {
- case bind(_,`c):
- el = el.append(c);
- };
- return makePlan(#<tuple(...el)>);
- case project(`x,`a):
- Tree tp = TypeInference.type_inference(x);
- match tp {
- case XML:
- return makePlan(#<call(XMLchildren,`(new StringLeaf(a.toString())),`x)>);
- case `S(XML):
- if (is_collection(S))
- return makePlan(#<call(XMLchildren,`(new StringLeaf(a.toString())),`x)>);
- };
- match TypeInference.expand(tp) {
- case record(...bl):
- int i = 0;
- for ( Tree b: bl )
- match b {
- case bind(`c,_):
- if (a.equals(c))
- return makePlan(#<nth(`x,`i)>);
- else fail
- case _: i++;
- };
- case union(...tl):
- for ( Tree t: tl )
- match TypeInference.expand(t) {
- case _(record(...bl)):
- int i = 0;
- for ( Tree b: bl )
- match b {
- case bind(`c,_):
- if (a.equals(c))
- return makePlan(#<nth(union_value(`x),`i)>);
- else fail
- case _: i++;
- };
- case _(bag(tuple(string,`tv))):
- return #<map_index(`(makePlan(#<union_value(`x)>)),
- `(new StringLeaf(a.toString())))>;
- case `tt: error("wrong projection: "+e+" ("+tt+")");
- };
- case `S(`ttp):
- if (!is_collection(S))
- fail;
- match TypeInference.expand(ttp) {
- case tuple(string,`tv):
- return #<map_index(`(makePlan(x)),
- `(new StringLeaf(a.toString())))>;
- case record(...bl):
- Tree nv = new_var();
- int i = 0;
- for ( Tree b: bl )
- match b {
- case bind(`c,_):
- if (a.equals(c))
- return makePlan(#<cmap(lambda(`nv,nth(`nv,`i)),`x)>);
- else fail
- case _: i++;
- };
- case union(...tl):
- Tree nv = new_var();
- int j = 0;
- for ( Tree t: tl ) {
- match t {
- case `c(record(...bl)):
- int i = 0;
- for ( Tree b: bl )
- match b {
- case bind(`w,_):
- if (a.equals(w))
- return makePlan(#<cmap(lambda(`nv,if(call(eq,union_tag(`nv),`j),
- bag(nth(union_value(`nv),`i)),
- bag())),`x)>);
- else fail
- case _: i++;
- };
- case `c(bag(tuple(string,`tv))):
- return makePlan(#<cmap(lambda(`nv,if(call(eq,union_tag(`nv),`j),
- bag(map_index(union_value(`nv),
- `(new StringLeaf(a.toString())))),
- bag())),`x)>);
- case `tt: error("wrong projection: "+e+" ("+tt+")");
- };
- j++;
- }
- };
- case `t: error("wrong projection: "+e+" ("+t+")");
- };
- case typed(`x,`tp):
- if (tp.is_variable() && !tp.equals(#<string>) && MRContainer.type_code(tp.toString()) >= 0)
- return makePlan(#<call(coerce,`x,
- `(MRContainer.type_code(tp.toString())))>);
- else fail
- case index(`x,`n):
- match TypeInference.type_inference2(x) {
- case bag(tuple(`kt,`vt)):
- if (TypeInference.type_inference2(n).equals(kt))
- return #<map_index(`(makePlan(x)),
- `(makePlan(n)))>;
- case Bag(tuple(`kt,`vt)):
- if (TypeInference.type_inference2(n).equals(kt))
- return #<map_index(`(makePlan(x)),
- `(makePlan(n)))>;
- case union(...tl):
- for ( Tree t: tl )
- match TypeInference.expand(t) {
- case _(bag(tuple(`kt,`vt))):
- if (TypeInference.type_inference2(n).equals(kt))
- return #<map_index(`(makePlan(#<union_value(`x)>)),
- `(makePlan(n)))>;
- case _(list(`tp)):
- return #<index(`(makePlan(#<union_value(`x)>)),
- `(makePlan(n)))>;
- case `tt: error("wrong indexing: "+e+" ("+tt+")");
- };
- };
- return #<index(`(makePlan(x)),
- `(makePlan(n)))>;
- case call(count,mapReduce(`m,lambda(`vr,`br),`X,`o)):
- Tree nv = new_var();
- type_env.insert(nv.toString(),TypeInference.type_inference(vr));
- Tree nr = simplify(#<lambda(`nv,bag(call(count,`(subst(vr,nv,br)))))>);
- Tree plan = #<call(sum,mapReduce(`m,`nr,`X,false))>;
- return makePlan(plan);
- case call(`f,`u):
- for ( Tree monoid: monoids )
- match monoid {
- case `aggr(`mtp,`plus,`zero,`unit):
- if (aggr.equals(f.toString())) {
- Tree plan = makePlan(u);
- Tree nx = new_var();
- Tree np = new_var();
- Tree na = new_var();
- Tree tp = TypeInference.type_inference(e);
- type_env.insert(np.toString(),#<tuple(`tp,`tp)>);
- match TypeInference.type_inference(u) {
- case `T(`t):
- if (TypeInference.unify(mtp,t) == null)
- continue;
- type_env.insert(na.toString(),#<tuple(`tp,`t)>);
- type_env.insert(nx.toString(),t);
- };
- plus = makePlan(simplify_all(#<lambda(`np,apply(`plus,`np))>));
- Tree acc = makePlan(simplify_all(#<lambda(`na,apply(`plus,tuple(nth(`na,0),
- apply(`unit,nth(`na,1)))))>));
- zero = makePlan((f.equals(#<avg>)) ? zero : #<typed(`zero,`tp)>);
- match plan {
- case MapCombineReduce(`m,`c,`r,`s,_):
- plan = #<MapAggregateReduce(`m,`r,`acc,`zero,`s,false)>;
- case MapReduce(`m,`r,`s,_):
- plan = #<MapAggregateReduce(`m,`r,`acc,`zero,`s,false)>;
- case MapReduce2(`mx,`my,`r,`x,`y,_):
- plan = #<MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,false)>;
- case MapJoin(`kx,`ky,`r,`x,`y):
- plan = #<MapAggregateJoin(`kx,`ky,`r,`acc,`zero,`x,`y)>;
- case CrossProduct(`mx,`my,`r,`x,`y):
- plan = #<CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y)>;
- case cMap(`m,`s):
- plan = #<AggregateMap(`m,`acc,`zero,`s)>;
- case _:
- if (is_dataset_expr(u))
- plan = #<AggregateMap(lambda(`nx,bag(`nx)),`acc,`zero,`plan)>;
- else return #<aggregate(`acc,`zero,`plan)>;
- };
- if (is_dataset_expr(u))
- return #<Aggregate(`plus,`zero,`plan)>;
- else return #<aggregate(`plus,`zero,`plan)>;
- }
- };
- fail
- case call(plus,`x,`y):
- if (!is_dataset_expr(x) || !is_dataset_expr(y))
- fail;
- return #<Merge(`(makePlan(x)),`(makePlan(y)))>;
- case call(`f,...el):
- if (!f.is_variable())
- fail;
- Tree ret = data_constructors.lookup(f.toString());
- if (ret != null)
- match ret {
- case `v(`n,`tp):
- Tree p = (el.length()==1)
- ? makePlan(el.head())
- : makePlan(#<tuple(...el)>);
- return #<tagged_union(`n,`p)>;
- };
- ret = global_type_env.lookup(f.toString());
- if (ret != null)
- match ret {
- case arrow(_,_):
- Trees args = #[];
- for ( Tree a: el )
- args = args.append(makePlan(a));
- return #<apply(`f,tuple(...args))>;
- };
- Trees tps = #[];
- for ( Tree a: el )
- tps = tps.append(TypeInference.type_inference(a));
- int i = ClassImporter.find_method_number(f.toString(),tps);
- if (i < 0)
- error("Method "+f+tps+" has no implementation");
- Trees sig = ClassImporter.signature(i);
- Trees args = #[];
- for ( int j = 0; j < el.length(); j++ ) {
- Tree b = sig.nth(j+1);
- if (f.equals(#<coerce>) || b.equals(tps.nth(j)) || !b.is_variable()
- || b.equals(#<union>) || MRContainer.type_code(b.toString()) < 0)
- args = args.append(makePlan(el.nth(j)));
- else args = args.append(makePlan(#<call(coerce,`(el.nth(j)),
- `(MRContainer.type_code(b.toString())))>));
- };
- return #<callM(`f,`i,...args)>;
- case let(`v,`u,`body):
- if (true)
- fail; // disable
- body = makePlan(body);
- match TypeInference.type_inference(u) {
- case `S(_):
- // if a variable bound to a collection is used more than once in the body,
- // materialize the collection in memory
- if (is_collection(S) && occurences(v,body) > 1)
- body = #<let(`v,`(makePlan(#<call(materialize,`v)>)),`body)>;
- };
- return #<let(`v,`(makePlan(u)),`body)>;
- case function(tuple(...params),`outp,`body):
- boolean is_dataset = false;
- for ( Tree p: params )
- match p {
- case dataset(`v,`tp):
- is_dataset = true;
- };
- body = makePlan(body);
- return #<function(tuple(...params),`outp,`body)>;
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(makePlan(a));
- return #<`f(...bl)>;
- };
- return e;
- }
-
- /** convert an algebraic plan to a physical plan
- * @param plan algebraic plan
- * @return the physical plan
- */
- public static Tree physical_plan ( Tree plan ) {
- match plan {
- case MapReduce2(`mx,`my,`r,`x,`y,`o):
- return physical_plan(#<MapAggregateReduce2(`mx,`my,`r,null,null,`x,`y,`o)>);
- // convert a reduce-side join to a fragment-replicate join, if either of the join
- // inputs is small to fit in memory (dataset sizes are extracted from file statistics)
- case MapAggregateReduce2(`mx,`my,
- lambda(`v,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nx,1))),nth(`ny,0))),
- null,null,`x,`y,false):
- if (Config.noMapJoin)
- fail;
- if (!nx.equals(v) || !ny.equals(v) || occurences(v,b) > 0)
- fail;
- Tree X = new_var();
- Tree Y = new_var();
- Tree nv = new_var();
- Tree L1 = simplify_all(#<lambda(`nv,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nv,1))),bag(nth(`nv,0))))>);
- nv = new_var();
- Tree L2 = simplify_all(#<lambda(`nv,cmap(lambda(`vx,cmap(lambda(`vy,`b),bag(nth(`nv,0)))),nth(`nv,1)))>);
- nv = new_var();
- Tree L3 = #<lambda(`nv,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nv,0))),nth(`nv,1)))>;
- Tree cond1 = makePlan(#<call(leq,dataset_size(`Y),`(Config.mapjoin_size))>);
- Tree cond2 = makePlan(#<call(leq,dataset_size(`X),`(Config.mapjoin_size))>);
- Tree cond3 = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
- return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
- If(`cond1,
- MapJoin(`mx,`my,`L1,`X,`Y),
- If(`cond2,
- MapJoin(`my,`mx,`L2,`Y,`X),
- If(`cond3,
- MapAggregateReduce2(`my,`mx,`L3,null,null,`Y,`X,false),
- MapAggregateReduce2(`mx,`my,
- lambda(`v,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nx,1))),
- nth(`ny,0))),
- null,null,`X,`Y,false))))))>;
- case MapAggregateReduce2(`mx,`my,lambda(`v,`b),null,null,`x,`y,false):
- if (Config.noMapJoin)
- fail;
- Tree X = new_var();
- Tree Y = new_var();
- Tree nv = new_var();
- Tree L1 = simplify_all(subst(#<nth(`v,0)>,#<bag(nth(`nv,0))>,
- subst(#<nth(`v,1)>,#<nth(`nv,1)>,
- #<lambda(`nv,`b)>)));
- nv = new_var();
- Tree L2 = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
- subst(#<nth(`v,1)>,#<nth(`nv,0)>,
- #<lambda(`nv,`b)>));
- Tree cond1 = makePlan(#<call(leq,dataset_size(`Y),`(Config.mapjoin_size))>);
- Tree cond2 = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
- return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
- If(`cond1,
- MapJoin(`mx,`my,`L1,`X,`Y),
- If(`cond2,
- MapAggregateReduce2(`my,`mx,`L2,null,null,`Y,`X,false),
- MapAggregateReduce2(`mx,`my,lambda(`v,`b),null,null,`X,`Y,false)))))>;
- case MapAggregateReduce2(`mx,`my,lambda(`v,`b),`acc,`zero,`x,`y,false):
- Tree X = new_var();
- Tree Y = new_var();
- Tree nv = new_var();
- Tree nr = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
- subst(#<nth(`v,1)>,#<nth(`nv,0)>,b));
- if (!streamed_MapReduce2_reducer(#<lambda(`nv,`nr)>))
- fail;
- Tree cond = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
- return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
- If(`cond,
- MapAggregateReduce2(`my,`mx,lambda(`nv,`nr),`acc,`zero,`Y,`X,false),
- MapAggregateReduce2(`mx,`my,lambda(`v,`b),`acc,`zero,`X,`Y,false))))>;
- case CrossProduct(`mx,`my,`r,`x,`y):
- return physical_plan(#<CrossAggregateProduct(`mx,`my,`r,null,null,`x,`y)>);
- case CrossAggregateProduct(`mx,`my,lambda(`v,`b),`acc,`zero,`x,`y):
- Tree X = new_var();
- Tree Y = new_var();
- Tree nv = new_var();
- Tree nr = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
- subst(#<nth(`v,1)>,#<nth(`nv,0)>,b));
- Tree cond = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
- return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
- If(`cond,
- CrossAggregateProduct(`my,`mx,lambda(`nv,`nr),`acc,`zero,`Y,`X),
- CrossAggregateProduct(`mx,`my,lambda(`v,`b),`acc,`zero,`X,`Y))))>;
- case `f(...al):
- Trees bl = #[];
- for ( Tree a: al )
- bl = bl.append(physical_plan(a));
- return #<`f(...bl)>;
- };
- return plan;
- }
-}
[24/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Bag.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Bag.java b/core/src/main/java/org/apache/mrql/Bag.java
new file mode 100644
index 0000000..b092f30
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Bag.java
@@ -0,0 +1,578 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/**
+ * A sequence of MRData.
+ * There are 3 kinds of Bag implementations, which are converted at run-time, when necessary:
+ * 1) vector-based (materialized): used for small bags (when size is less than Config.max_materialized_bag);
+ * 2) stream-based: can be traversed only once; implemented as Java iterators;
+ * 3) spilled to a local file: can be accessed multiple times
+ */
+public class Bag extends MRData implements Iterable<MRData> {
+ private final static long serialVersionUID = 64629834894869L;
+ enum Modes { STREAMED, MATERIALIZED, SPILLED };
+ private transient Modes mode;
+ private transient ArrayList<MRData> content; // content of a materialized bag
+ private transient BagIterator iterator; // iterator for a streamed bag
+ private transient boolean consumed; // true, if the stream has already been used
+ private transient String path; // local path that contains the spilled bag
+ private transient SequenceFile.Writer writer; // the file writer for spiled bags
+
+ /**
+ * create an empty bag as an ArrayList
+ */
+ public Bag () {
+ mode = Modes.MATERIALIZED;
+ content = new ArrayList<MRData>();
+ }
+
+ /**
+ * create an empty bag as an ArrayList with a given capacity
+ * @param size initial capacity
+ */
+ public Bag ( final int size ) {
+ mode = Modes.MATERIALIZED;
+ content = new ArrayList<MRData>(size);
+ }
+
+ /**
+ * in-memory Bag construction (an ArrayList) initialized with data
+ * @param as a vector of MRData to insert in the Bag
+ */
+ public Bag ( final MRData ...as ) {
+ mode = Modes.MATERIALIZED;
+ content = new ArrayList<MRData>(as.length);
+ for ( MRData a: as )
+ content.add(a);
+ }
+
+ /**
+ * in-memory Bag construction (an ArrayList) initialized with data
+ * @param as a vector of MRData to insert in the Bag
+ */
+ public Bag ( final List<MRData> as ) {
+ mode = Modes.MATERIALIZED;
+ content = new ArrayList<MRData>(as.size());
+ for ( MRData a: as )
+ content.add(a);
+ }
+
+ /**
+ * lazy construction (stream-based) of a Bag
+ * @param i the Iterator that generates the Bag elements
+ */
+ public Bag ( final BagIterator i ) {
+ mode = Modes.STREAMED;
+ iterator = i;
+ consumed = false;
+ }
+
+ /** is the Bag stored in an ArrayList? */
+ public boolean materialized () {
+ return mode == Modes.MATERIALIZED;
+ }
+
+ /** is the Bag stream-based? */
+ public boolean streamed () {
+ return mode == Modes.STREAMED;
+ }
+
+ /** is the Bag spilled into a file? */
+ public boolean spilled () {
+ return mode == Modes.SPILLED;
+ }
+
+ /** return the Bag size (cache it in memory if necessary) */
+ public int size () {
+ if (materialized())
+ return content.size();
+ if (streamed() && consumed)
+ throw new Error("*** The collection stream has already been consumed");
+ int i = 0;
+ for ( MRData e: this )
+ i++;
+ if (streamed())
+ consumed = true;
+ return i;
+ }
+
+ /** trim the ArrayList that caches the Bag */
+ public void trim () {
+ if (materialized())
+ content.trimToSize();
+ }
+
+ /** get the n'th element of a Bag (cache it in memory if necessary)
+ * @param n the index
+ * @return the n'th element
+ */
+ public MRData get ( final int n ) {
+ if (materialized())
+ if (n < size())
+ return content.get(n);
+ else throw new Error("List index out of range: "+n);
+ if (streamed() && consumed)
+ throw new Error("*** The collection stream has already been consumed");
+ int i = 0;
+ for ( MRData e: this )
+ if (i++ == n)
+ return e;
+ if (streamed())
+ consumed = true;
+ throw new Error("Cannot retrieve the "+n+"th element of a sequence");
+ }
+
+ /** replace the n'th element of a Bag with a new value
+ * @param n the index
+ * @param value the new value
+ * @return the Bag
+ */
+ public Bag set ( final int n, final MRData value ) {
+ if (!materialized())
+ throw new Error("Cannot replace an element of a non-materialized sequence");
+ content.set(n,value);
+ return this;
+ }
+
+ /** add a new value to a Bag (cache it in memory if necessary)
+ * @param x the new value
+ */
+ public void add ( final MRData x ) {
+ materialize();
+ if (!spilled() && Config.hadoop_mode
+ && size() >= Config.max_materialized_bag)
+ spill();
+ if (spilled())
+ try {
+ if (writer == null) { // writer was closed earlier for reading
+ FileSystem fs = FileSystem.getLocal(Plan.conf);
+ writer = SequenceFile.createWriter(fs,Plan.conf,new Path(path),
+ MRContainer.class,NullWritable.class,
+ SequenceFile.CompressionType.NONE);
+ System.err.println("*** Appending elements to a spilled Bag: "+path);
+ };
+ writer.append(new MRContainer(x),NullWritable.get());
+ } catch (IOException e) {
+ throw new Error("Cannot append an element to a spilled Bag: "+path);
+ }
+ else content.add(x);
+ }
+
+ /** add a new value to a Bag (cache it in memory if necessary)
+ * @param x the new value
+ * @return the Bag
+ */
+ public Bag add_element ( final MRData x ) {
+ add(x);
+ return this;
+ }
+
+ /** add the elements of a Bag to the end of this Bag
+ * @param b the Bag whose elements are copied
+ * @return the Bag
+ */
+ public Bag addAll ( final Bag b ) {
+ for ( MRData e: b )
+ add(e);
+ return this;
+ }
+
+ /** make this Bag empty (cache it in memory if necessary) */
+ public void clear () {
+ if (materialized())
+ content.clear();
+ else if (streamed()) {
+ if (writer != null)
+ try {
+ writer.close();
+ } catch (IOException ex) {
+ throw new Error(ex);
+ };
+ writer = null;
+ path = null;
+ mode = Modes.MATERIALIZED;
+ content = new ArrayList<MRData>(100);
+ };
+ mode = Modes.MATERIALIZED;
+ content = new ArrayList<MRData>();
+ }
+
+ /** cache the Bag to an ArrayList when is absolutely necessary */
+ public void materialize () {
+ if (materialized() || spilled())
+ return;
+ Iterator<MRData> iter = iterator();
+ mode = Modes.MATERIALIZED;
+ writer = null;
+ path = null;
+ content = new ArrayList<MRData>(100);
+ while ( iter.hasNext() )
+ add(iter.next());
+ if (materialized()) // it may have been spilled
+ content.trimToSize();
+ iterator = null;
+ }
+
+ private static Random random_generator = new Random();
+
+ private static String new_path ( FileSystem fs ) throws IOException {
+ Path p;
+ do {
+ p = new Path("file://"+Config.tmpDirectory+"/mrql"+(random_generator.nextInt(1000000)));
+ } while (p.getFileSystem(Plan.conf).exists(p));
+ String path = p.toString();
+ Plan.temporary_paths.add(path);
+ return path;
+ }
+
+ /** spill the Bag to a local file */
+ private void spill () {
+ if (!spilled() && Config.hadoop_mode)
+ try {
+ if (Plan.conf == null)
+ Plan.conf = Evaluator.evaluator.new_configuration();
+ final FileSystem fs = FileSystem.getLocal(Plan.conf);
+ path = new_path(fs);
+ System.err.println("*** Spilling a Bag to a local file: "+path);
+ final Path p = new Path(path);
+ writer = SequenceFile.createWriter(fs,Plan.conf,new Path(path),
+ MRContainer.class,NullWritable.class,
+ SequenceFile.CompressionType.NONE);
+ for ( MRData e: this )
+ writer.append(new MRContainer(e),NullWritable.get());
+ mode = Modes.SPILLED;
+ content = null;
+ iterator = null;
+ } catch (Exception e) {
+ throw new Error("Cannot spill a Bag to a local file");
+ }
+ }
+
+ /**
+ * sort the Bag (cache it in memory if necessary).
+ * If the Bag was spilled during caching, use external sorting
+ */
+ public void sort () {
+ materialize();
+ if (spilled()) // if it was spilled during materialize()
+ try { // use external sorting
+ if (writer != null)
+ writer.close();
+ FileSystem fs = FileSystem.getLocal(Plan.conf);
+ SequenceFile.Sorter sorter
+ = new SequenceFile.Sorter(fs,new Plan.MRContainerKeyComparator(),
+ MRContainer.class,NullWritable.class,Plan.conf);
+ String out_path = new_path(fs);
+ System.err.println("*** Using external sorting on a spilled bag "+path+" -> "+out_path);
+ sorter.setMemory(64*1024*1024);
+ sorter.sort(new Path(path),new Path(out_path));
+ path = out_path;
+ writer = null;
+ } catch (Exception ex) {
+ throw new Error("Cannot sort a spilled bag");
+ }
+ else Collections.sort(content);
+ }
+
+ /** return the Bag Iterator */
+ public Iterator<MRData> iterator () {
+ if (spilled())
+ try {
+ if (writer != null)
+ writer.close();
+ writer = null;
+ return new BagIterator () {
+ final FileSystem fs = FileSystem.getLocal(Plan.conf);
+ final SequenceFile.Reader reader = new SequenceFile.Reader(fs,new Path(path),Plan.conf);
+ final MRContainer key = new MRContainer();
+ final NullWritable value = NullWritable.get();
+ MRData data;
+ public boolean hasNext () {
+ try {
+ if (!reader.next(key,value)) {
+ reader.close();
+ return false;
+ };
+ data = key.data();
+ return true;
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from a spilled Bag");
+ }
+ }
+ public MRData next () {
+ return data;
+ }
+ };
+ } catch (IOException e) {
+ throw new Error("Cannot collect values from a spilled Bag");
+ }
+ else if (materialized())
+ return content.iterator();
+ else {
+ if (consumed) // this should never happen
+ throw new Error("*** The collection stream has already been consumed");
+ consumed = true;
+ return iterator;
+ }
+ }
+
+ /** cache MRData in memory by caching all Bags at any place and depth in MRData */
+ public void materializeAll () {
+ materialize();
+ for (MRData e: this)
+ e.materializeAll();
+ }
+
+ /** concatenate the elements of a given Bag to the elements of this Bag.
+ * Does not change either Bag
+ * @param s the given Bag
+ * @return a new Bag
+ */
+ public Bag union ( final Bag s ) {
+ final Iterator<MRData> i1 = iterator();
+ final Iterator<MRData> i2 = s.iterator();
+ return new Bag(new BagIterator () {
+ boolean first = true;
+ public boolean hasNext () {
+ if (first)
+ if (i1.hasNext())
+ return true;
+ else {
+ first = false;
+ return i2.hasNext();
+ }
+ else return i2.hasNext();
+ }
+ public MRData next () {
+ if (first)
+ return i1.next();
+ else return i2.next();
+ }
+ });
+ }
+
+ /** does this Bag contain an element?
+ * Cache this Bag in memory befor tetsing if necessary
+ * @param x the element to find
+ */
+ public boolean contains ( final MRData x ) {
+ if (materialized())
+ return content.contains(x);
+ if (streamed() && consumed)
+ throw new Error("*** The collection stream has already been consumed");
+ for ( MRData e: this )
+ if (x.equals(e))
+ return true;
+ if (streamed())
+ consumed = true;
+ return false;
+ }
+
+ /** if this Bag is a Map from keys to values (a Bag of (key,value) pairs),
+ * find the value with the given key; raise an error if not found
+ * @param key the search key
+ * @return the value associated with the key
+ */
+ public MRData map_find ( final MRData key ) {
+ if (streamed() && consumed)
+ throw new Error("*** The collection stream has already been consumed");
+ for ( MRData e: this ) {
+ Tuple p = (Tuple) e;
+ if (key.equals(p.first()))
+ return p.second();
+ };
+ if (streamed())
+ consumed = true;
+ throw new Error("key "+key+" not found in map");
+ }
+
+ /** if this Bag is a Map from keys to values (a Bag of (key,value) pairs),
+ * does it contain a given key?
+ * @param key the search key
+ */
+ public boolean map_contains ( final MRData key ) {
+ if (streamed() && consumed)
+ throw new Error("*** The collection stream has already been consumed");
+ for ( MRData e: this )
+ if (key.equals(((Tuple)e).first()))
+ return true;
+ if (streamed())
+ consumed = true;
+ return false;
+ }
+
+ /** the output serializer for Bag.
+ * Stream-based Bags are serialized lazily (without having to cache the Bag in memory)
+ */
+ final public void write ( DataOutput out ) throws IOException {
+ if (materialized()) {
+ out.writeByte(MRContainer.BAG);
+ WritableUtils.writeVInt(out,size());
+ for ( MRData e: this )
+ e.write(out);
+ } else {
+ out.writeByte(MRContainer.LAZY_BAG);
+ for ( MRData e: this )
+ e.write(out);
+ out.writeByte(MRContainer.END_OF_LAZY_BAG);
+ }
+ }
+
+ /** the input serializer for Bag */
+ final public static Bag read ( DataInput in ) throws IOException {
+ int n = WritableUtils.readVInt(in);
+ Bag bag = new Bag(n);
+ for ( int i = 0; i < n; i++ )
+ bag.add(MRContainer.read(in));
+ return bag;
+ }
+
+ /** a lazy input serializer for a Bag (it doesn't need to cache a Bag in memory) */
+ public static Bag lazy_read ( final DataInput in ) throws IOException {
+ Bag bag = new Bag(100);
+ MRData data = MRContainer.read(in);
+ while (data != MRContainer.end_of_lazy_bag) {
+ bag.add(data);
+ data = MRContainer.read(in);
+ };
+ if (bag.materialized())
+ bag.content.trimToSize();
+ return bag;
+ }
+
+ /** the input serializer for Bag */
+ public void readFields ( DataInput in ) throws IOException {
+ int n = WritableUtils.readVInt(in);
+ mode = Modes.MATERIALIZED;
+ iterator = null;
+ path = null;
+ writer = null;
+ if (content == null)
+ content = new ArrayList<MRData>(n);
+ else {
+ content.clear();
+ content.ensureCapacity(n);
+ };
+ for ( int i = 0; i < n; i++ )
+ add(MRContainer.read(in));
+ }
+
+ private void writeObject ( ObjectOutputStream out ) throws IOException {
+ materialize();
+ WritableUtils.writeVInt(out,size());
+ for ( MRData e: this )
+ e.write(out);
+ }
+
+ private void readObject ( ObjectInputStream in ) throws IOException, ClassNotFoundException {
+ int n = WritableUtils.readVInt(in);
+ mode = Modes.MATERIALIZED;
+ iterator = null;
+ path = null;
+ writer = null;
+ content = new ArrayList<MRData>(n);
+ for ( int i = 0; i < n; i++ )
+ add(MRContainer.read(in));
+ }
+
+ private void readObjectNoData () throws ObjectStreamException { };
+
+ /** compare this Bag with a given Bag by comparing their associated elements */
+ public int compareTo ( MRData x ) {
+ Bag xt = (Bag)x;
+ Iterator<MRData> xi = xt.iterator();
+ Iterator<MRData> yi = iterator();
+ while ( xi.hasNext() && yi.hasNext() ) {
+ int c = xi.next().compareTo(yi.next());
+ if (c < 0)
+ return -1;
+ else if (c > 0)
+ return 1;
+ };
+ if (xi.hasNext())
+ return -1;
+ else if (yi.hasNext())
+ return 1;
+ else return 0;
+ }
+
+ /** compare this Bag with a given Bag by comparing their associated elements */
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ try {
+ int xn = WritableComparator.readVInt(x,xs);
+ int xx = WritableUtils.decodeVIntSize(x[xs]);
+ int yn = WritableComparator.readVInt(y,ys);
+ int yy = WritableUtils.decodeVIntSize(y[ys]);
+ for ( int i = 0; i < xn && i < yn; i++ ) {
+ int k = MRContainer.compare(x,xs+xx,xl-xx,y,ys+yy,yl-yy,size);
+ if (k != 0)
+ return k;
+ xx += size[0];
+ yy += size[0];
+ };
+ size[0] = xx+1;
+ if (xn > yn)
+ return 1;
+ if (xn < yn)
+ return -1;
+ return 0;
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ /** is this Bag equal to another Bag (order is important) */
+ public boolean equals ( Object x ) {
+ if (!(x instanceof Bag))
+ return false;
+ Bag xt = (Bag) x;
+ Iterator<MRData> xi = xt.iterator();
+ Iterator<MRData> yi = iterator();
+ while ( xi.hasNext() && yi.hasNext() )
+ if ( !xi.next().equals(yi.next()) )
+ return false;
+ return xi.hasNext() || yi.hasNext();
+ }
+
+ /** the hash code of this Bag is the XOR of the hash code of its elements */
+ public int hashCode () {
+ int h = 127;
+ for ( MRData e: this )
+ h ^= e.hashCode();
+ return Math.abs(h);
+ }
+
+ /** show the first few Bag elements (controlled by -bag_print) */
+ public String toString () {
+ materialize();
+ StringBuffer b = new StringBuffer("{ ");
+ int i = 0;
+ for ( MRData e: this )
+ if ( i++ < Config.max_bag_size_print )
+ b.append(((i>1)?", ":"")+e);
+ else return b.append(", ... }").toString();
+ return b.append(" }").toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/BagIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/BagIterator.java b/core/src/main/java/org/apache/mrql/BagIterator.java
new file mode 100644
index 0000000..0fd7354
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/BagIterator.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.util.Iterator;
+
+/** an Iterator over Bags */
+abstract public class BagIterator implements Iterator<MRData> {
+ public void remove () {
+ throw new Error("Bag deletions are not permitted");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/BinaryDataSource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/BinaryDataSource.java b/core/src/main/java/org/apache/mrql/BinaryDataSource.java
new file mode 100644
index 0000000..d4338ec
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/BinaryDataSource.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.hadoop.conf.Configuration;
+
+
+/** A DataSource used for storing intermediate results and data dumps */
+final public class BinaryDataSource extends DataSource {
+ BinaryDataSource ( int source_num, String path, Configuration conf ) {
+ super(source_num,path,Evaluator.evaluator.binaryInputFormat(),conf);
+ }
+
+ BinaryDataSource ( String path, Configuration conf ) {
+ super(-1,path,Evaluator.evaluator.binaryInputFormat(),conf);
+ }
+
+ public String toString () {
+ return "Binary"+separator+source_num+separator+path;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/ClassImporter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/ClassImporter.java b/core/src/main/java/org/apache/mrql/ClassImporter.java
new file mode 100644
index 0000000..8efc1ae
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/ClassImporter.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.lang.reflect.Method;
+import org.apache.mrql.gen.*;
+import java.util.*;
+
+/** imports external Java methods into MRQL */
+final public class ClassImporter {
+ final static boolean trace_imported_methods = false;
+
+ final static String[] object_methods
+ = { "hashCode", "getClass", "wait", "equals", "toString", "notify", "notifyAll" };
+
+ static Vector<MethodInfo> methods = new Vector<MethodInfo>();
+
+ public static void load_classes () {
+ if (methods == null)
+ methods = new Vector<MethodInfo>();
+ if (methods.size() == 0) {
+ importClass("org.apache.mrql.SystemFunctions");
+ //****** import your classes with user-defined functions here
+ }
+ }
+
+ private static boolean object_method ( String s ) {
+ for (int i = 0; i < object_methods.length; i++)
+ if (object_methods[i].equals(s))
+ return true;
+ return false;
+ }
+
+ private static Tree getType ( Class<?> c ) {
+ String cn = c.getCanonicalName();
+ Class<?>[] inf = c.getInterfaces();
+ if (cn.equals("org.apache.mrql.MRData"))
+ return new VariableLeaf("any");
+ if (cn.startsWith("org.apache.mrql.MR_"))
+ return new VariableLeaf(cn.substring(19));
+ if (cn.equals("org.apache.mrql.Bag"))
+ return new Node("bag",new Trees(new VariableLeaf("any")));
+ if (cn.equals("org.apache.mrql.Inv"))
+ return new VariableLeaf("any");
+ if (cn.equals("org.apache.mrql.Union"))
+ return new VariableLeaf("union");
+ if (cn.equals("org.apache.mrql.Lambda"))
+ return new VariableLeaf("any");
+ if (inf.length > 0 && inf[0].equals("org.apache.mrql.MRData"))
+ return new VariableLeaf("any");
+ throw new Error("Unsupported type in imported method: "+cn);
+ }
+
+ private static Trees signature ( Method m ) {
+ Class<?> co = m.getReturnType();
+ Class<?>[] cs = m.getParameterTypes();
+ Trees as = new Trees(getType(co));
+ for (int i = 0; i < cs.length; i++)
+ as = as.append(getType(cs[i]));
+ return as;
+ }
+
+ public static String method_name ( int method_number ) {
+ return methods.get(method_number).name;
+ }
+
+ public static Trees signature ( int method_number ) {
+ return methods.get(method_number).signature;
+ }
+
+ /** import all Java methods from a given Java class */
+ public static void importClass ( String class_name ) {
+ try {
+ Method[] ms = Class.forName(class_name).getMethods();
+ Vector<MethodInfo> mv = new Vector<MethodInfo>();
+ for (int i = 0; i < ms.length; i++)
+ if (!object_method(ms[i].getName()) && ms[i].getModifiers() == 9)
+ try {
+ Trees sig = signature(ms[i]);
+ MethodInfo m = new MethodInfo(ms[i].getName(),sig,ms[i]);
+ mv.add(m);
+ methods.add(m);
+ } catch ( Exception e ) {
+ System.out.println("Warning: method "+ms[i].getName()+" cannot be imported");
+ System.out.println(e);
+ throw new Error("");
+ };
+ Collections.sort(methods);
+ if (Translator.functions == null)
+ Translator.functions = Trees.nil;
+ for ( MethodInfo m: methods )
+ Translator.functions = Translator.functions.append(new Node(m.name,m.signature));
+ if (trace_imported_methods) {
+ System.out.print("Importing methods: ");
+ for (int i = 0; i < mv.size(); i++ )
+ System.out.print(mv.get(i).name+mv.get(i).signature.tail()
+ +":"+mv.get(i).signature.head()+" ");
+ System.out.println();
+ }
+ } catch (ClassNotFoundException x) {
+ throw new Error("Undefined class: "+class_name);
+ }
+ }
+
+ /** import a Java method with a given name from a given Java class */
+ public static void importMethod ( String class_name, String method_name ) {
+ try {
+ Method[] ms = Class.forName(class_name).getMethods();
+ MethodInfo m = null;
+ for (int i = 0; i < ms.length; i++)
+ if (ms[i].getName().equals(method_name)
+ && !object_method(ms[i].getName()) && ms[i].getModifiers() == 9) {
+ Trees sig = signature(ms[i]);
+ m = new MethodInfo(ms[i].getName(),sig,ms[i]);
+ Translator.functions = Translator.functions.append(new Node(ms[i].getName(),sig));
+ break;
+ };
+ if (m == null)
+ throw new Error("No such method: "+method_name);
+ methods.add(m);
+ Collections.sort(methods);
+ if (trace_imported_methods)
+ System.out.println("Importing method: "+m.name+m.signature.tail()
+ +":"+m.signature.head()+" ");
+ } catch (ClassNotFoundException x) {
+ throw new Error("Undefined class: "+class_name);
+ }
+ }
+
+ public static void print_methods () {
+ for (int i = 0; i < methods.size(); i++ ) {
+ MethodInfo m = methods.get(i);
+ System.out.print(" "+m.name+":"+m.signature.tail()+"->"+m.signature.head());
+ };
+ }
+
+ /** return the method specification of a system method with a given name over some expressions;
+ * When the method is overloaded, find the most specific (in terms of arg subtyping)
+ * @param method_name the given method name
+ * @param args the method expressions
+ * @return the method specification
+ */
+ public static Tree find_method ( String method_name, Trees args ) {
+ for (int i = 0; i < methods.size(); i++ ) {
+ MethodInfo m = methods.get(i);
+ if (m.name.equals(method_name) && TypeInference.subtype(args,m.signature.tail()))
+ return m.signature.head();
+ };
+ return null;
+ }
+
+ /** return the method number of a system method with a given name over some expressions;
+ * When the method is overloaded, find the most specific (in terms of arg subtyping)
+ * @param method_name the given method name
+ * @param args the method expressions
+ * @return the method number
+ */
+ public static int find_method_number ( String method_name, Trees args ) {
+ for (int i = 0; i < methods.size(); i++ ) {
+ MethodInfo m = methods.get(i);
+ if (m.name.equals(method_name) && TypeInference.subtype(args,m.signature.tail()))
+ return i;
+ };
+ return -1;
+ }
+
+ /** call a system method with a given number over MRData
+ * @param method_number the method number
+ * @param args in input arguments
+ * @return the result of invoking this method over the args
+ */
+ public static MRData call ( int method_number, MRData... args ) {
+ if (method_number < 0 || method_number >= methods.size())
+ throw new Error("Run-time error (unknown method name)");
+ MethodInfo m = methods.get(method_number);
+ try {
+ return (MRData)m.method.invoke(null,(Object[])args);
+ } catch (Exception e) {
+ Tuple t = new Tuple(args.length);
+ for ( int i = 0; i < args.length; i++ )
+ t.set(i,args[i]);
+ System.err.println("Run-time error in method call: "+m.name+t+" of type "
+ +m.signature.tail()+"->"+m.signature.head());
+ throw new Error(e.toString());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Compiler.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Compiler.gen b/core/src/main/java/org/apache/mrql/Compiler.gen
new file mode 100644
index 0000000..668cdc8
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Compiler.gen
@@ -0,0 +1,584 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import javax.tools.*;
+import javax.tools.JavaCompiler.CompilationTask;
+import java.lang.reflect.*;
+import java.util.*;
+import java.net.*;
+import java.util.jar.*;
+import org.apache.hadoop.io.WritableComparable;
+
+
+/** compilation of MRQL expressions to Java code and then to Java bytecode */
+final public class Compiler extends Translator {
+ static JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
+ static DiagnosticCollector<JavaFileObject> diagnostics = new DiagnosticCollector<JavaFileObject>();
+ final static String tmp_dir = "/tmp/mrql_jar_"+System.getProperty("user.name");
+ public static String jar_path;
+ static Trees in_memory_repeat_vars = #[];
+ static int lambda_num = 0;
+ static int user_functions_num = 0;
+
+ /** Compile the MRQL functional arguments into Java bytecode */
+ final private static class JavaSourceFromString extends SimpleJavaFileObject {
+ final String code;
+
+ JavaSourceFromString ( String name, String code ) {
+ super(URI.create("string:///org/apache/mrql/" + name.replace('.','/') + Kind.SOURCE.extension),Kind.SOURCE);
+ this.code = code;
+ }
+
+ @Override
+ public CharSequence getCharContent ( boolean ignoreEncodingErrors ) {
+ return code;
+ }
+ }
+
+ private static String new_lambda_name () {
+ return "MRQL_Lambda_"+(lambda_num++);
+ }
+
+ private static void add2jar ( File source, int offset, JarOutputStream target ) throws IOException {
+ if (source.isDirectory()) {
+ String name = source.getPath();
+ if (name.length() > offset) {
+ JarEntry entry = new JarEntry(name.substring(offset));
+ entry.setTime(source.lastModified());
+ target.putNextEntry(entry);
+ target.closeEntry();
+ };
+ for ( File nestedFile: source.listFiles() )
+ add2jar(nestedFile,offset,target);
+ } else {
+ JarEntry entry = new JarEntry(source.getPath().substring(offset));
+ entry.setTime(source.lastModified());
+ target.putNextEntry(entry);
+ BufferedInputStream in = new BufferedInputStream(new FileInputStream(source));
+ byte[] buffer = new byte[1024];
+ int count = 1;
+ while (count > 0) {
+ count = in.read(buffer);
+ if (count > 0)
+ target.write(buffer,0,count);
+ };
+ target.closeEntry();
+ in.close();
+ }
+ }
+
+ private static void remove ( File file ) throws IOException {
+ if (file.isDirectory())
+ for ( File nestedFile: file.listFiles() )
+ remove(nestedFile);
+ file.delete();
+ }
+
+ public static void clean () throws IOException {
+ in_memory_repeat_vars = #[];
+ remove(new File(tmp_dir));
+ }
+
+ final private static Tree compile ( Tree e, StringBuffer out ) throws Exception {
+ match e {
+ case repeat(lambda(`v,`body),`s,`n):
+ in_memory_repeat_vars = in_memory_repeat_vars.cons(v);
+ return #<repeat(lambda(`v,`(compile(body,out))),`(compile(s,out)),`n)>;
+ case `f(...al):
+ if (!plans_with_distributed_lambdas.member(#<`f>))
+ fail;
+ Trees nl = #[];
+ for ( Tree a: al)
+ match Interpreter.closure(a,Interpreter.global_env) {
+ case lambda(`v,`body):
+ String fname = new_lambda_name();
+ StringBuffer sb = new StringBuffer(1000);
+ sb.append("final class "+fname+" extends Function {\n");
+ sb.append(fname+" () {}\n");
+ sb.append("final public MRData eval ( final MRData "+v
+ +" ) { return "+compileE(body)+"; }\n}\n");
+ out.append(sb);
+ nl = nl.append(#<compiled(`fname,`a)>);
+ case _: nl = nl.append(compile(a,out));
+ };
+ return #<`f(...nl)>;
+ case `f(...al):
+ Trees nl = #[];
+ for ( Tree a: al)
+ nl = nl.append(compile(a,out));
+ return #<`f(...nl)>;
+ };
+ return e;
+ }
+
+ private static StringBuffer out;
+
+ /** compile the functional arguments of the MRQL operators using the Java compiler
+ * @param query the expression to compile
+ * @return the query with all functional arguments compiled to Java bytecode
+ */
+ final public static Tree compile ( Tree query ) {
+ try {
+ user_functions_num = lambda_num++;
+ // remove the old jar
+ if (jar_path != null)
+ remove(new File(jar_path));
+ jar_path = tmp_dir+"/mrql_args_"+(new Random().nextInt(1000000))+".jar";
+ out = new StringBuffer(1000);
+ out.append("package org.apache.mrql;\n");
+ Tree nq = compile(query,out);
+ StringBuffer sb = new StringBuffer(1000);
+ for ( String f: global_functions )
+ match global_functions.lookup(f) {
+ case function(tuple(...params),`otp,`body):
+ sb.append("final public static "+get_MR_type(otp)+" "+f);
+ if (params.is_empty())
+ sb.append(" ()");
+ else {
+ match params.head() {
+ case bind(`v,`tp):
+ sb.append(" ( final "+get_MR_type(tp)+" "+v);
+ };
+ for ( Tree var: params.tail() )
+ match var {
+ case bind(`v,`tp):
+ sb.append(", final "+get_MR_type(tp)+" "+v);
+ }
+ sb.append(" ) { return ("+get_MR_type(otp)+")");
+ sb.append(compileE(body));
+ sb.append("; }\n");
+ }
+ };
+ out.append("final class UserFunctions_"+user_functions_num+" {\n");
+ out.append(sb);
+ out.append("}\n");
+ String code = out.toString();
+ //System.out.println(code);
+ JavaFileObject file = new JavaSourceFromString("UserFunctions_"+user_functions_num,code);
+ Iterable<? extends JavaFileObject> compilationUnits = Arrays.asList(file);
+ List<String> optionList = new ArrayList<String>();
+ (new File(tmp_dir)).mkdir();
+ String dir = tmp_dir+"/classes_"+(new Random().nextInt(1000000));
+ File fdir = new File(dir);
+ fdir.mkdir();
+ ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+ String classpath = jar_path;
+ String separator = System.getProperty("path.separator");
+ for ( URL url: ((URLClassLoader) classLoader).getURLs() )
+ classpath += separator+url.getFile();
+ // use hadoop core jar
+ classpath += separator + WritableComparable.class.getProtectionDomain().getCodeSource().getLocation().toString();
+ optionList.addAll(Arrays.asList("-classpath",classpath));
+ optionList.addAll(Arrays.asList("-d",dir));
+ CompilationTask task = compiler.getTask(null,null,diagnostics,optionList,null,compilationUnits);
+ boolean success = task.call();
+ if (!success)
+ for ( Diagnostic d: diagnostics.getDiagnostics() )
+ System.err.println("*** Compilation error at line "+d.getLineNumber()+" position "
+ +d.getColumnNumber()+": "+d.getMessage(Locale.US));
+ Manifest manifest = new Manifest();
+ manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION,"1.0");
+ JarOutputStream target = new JarOutputStream(new FileOutputStream(jar_path),manifest);
+ add2jar(new File(dir+"/"),dir.length()+1,target);
+ target.close();
+ remove(fdir);
+ return nq;
+ } catch (Exception e) {
+ System.err.println("*** Warning: Unable to compile the query:\n"+query);
+ if (Config.trace)
+ e.printStackTrace(System.err);
+ return query;
+ }
+ }
+
+ /** load the Java class of the anonymous function with name lambda_name */
+ final public static Function compiled ( ClassLoader cl, String lambda_name ) throws Exception {
+ URL[] urls = ((URLClassLoader) cl).getURLs();
+ URL[] new_urls = new URL[urls.length+1];
+ for ( int i = 0; i < urls.length; i++ )
+ new_urls[i+1] = urls[i];
+ new_urls[0] = new URL("file://"+jar_path);
+ URLClassLoader loader = new URLClassLoader(new_urls,cl);
+ Class c = loader.loadClass("org.apache.mrql."+lambda_name);
+ Constructor cc = c.getDeclaredConstructors()[0];
+ cc.setAccessible(true);
+ return (Function)cc.newInstance();
+ }
+
+ /** The Java type of an MRQL type */
+ private static String get_MR_type ( Tree type ) {
+ match type {
+ case boolean: return "MR_bool";
+ case byte: return "MR_byte";
+ case short: return "MR_short";
+ case int: return "MR_int";
+ case long: return "MR_long";
+ case float: return "MR_float";
+ case double: return "MR_double";
+ case char: return "MR_char";
+ case string: return "MR_string";
+ case union: return "Union";
+ case bag(...): return "Bag";
+ case list(...): return "Bag";
+ };
+ return "MRData";
+ }
+
+ private static Trees remove_duplicates ( Trees al ) {
+ if (al.is_empty())
+ return al;
+ Trees el = remove_duplicates(al.tail());
+ if (el.member(al.head()))
+ return el;
+ else return el.cons(al.head());
+ }
+
+ /** lambda lifting: generate Java code from an anonymous function */
+ private static String compilef ( String v, Tree body ) throws Exception {
+ String fname = new_lambda_name();
+ Trees free_vars = remove_duplicates(free_variables(body,#[`v]));
+ StringBuffer sb = new StringBuffer(1000);
+ sb.append("final class "+fname+" extends Function {\n");
+ for ( Tree var: free_vars )
+ sb.append("MRData "+var+"; ");
+ sb.append("\npublic "+fname+" (");
+ if (free_vars.is_empty())
+ sb.append(") {}\n");
+ else {
+ sb.append(" MRData "+free_vars.head());
+ for ( Tree var: free_vars.tail() )
+ sb.append(", MRData "+var);
+ sb.append(" ) { ");
+ for ( Tree var: free_vars )
+ sb.append("this."+var+" = "+var+"; ");
+ sb.append("}\n");
+ };
+ sb.append("final public MRData eval ( final MRData "+v
+ +" ) { return "+compileE(body)+"; }\n}\n");
+ out.append(sb);
+ String s = "new "+fname+"(";
+ if (!free_vars.is_empty()) {
+ s += free_vars.head();
+ for ( Tree var: free_vars.tail() )
+ s += ","+var;
+ };
+ return s+")";
+ }
+
+ private static String compileF ( Tree fnc ) throws Exception {
+ match fnc {
+ case lambda(`v,`b):
+ return compilef(v.toString(),b);
+ case compiled(`f,`lm):
+ // recompile the function
+ String s = compileF(lm);
+ ((Node)fnc).children().head = new VariableLeaf(s.substring(4,s.indexOf("("))); //destructive
+ return s;
+ case function(tuple(...params),`tp,`body):
+ String ret = "new Lambda(new Function () { "
+ +"final public MRData eval ( final MRData _x ) { ";
+ for ( int i = 0; i < params.length(); i++ )
+ match params.nth(i) {
+ case bind(`v,_):
+ ret += "final MRData "+v+" = ((Tuple)_x).get("+i+"); ";
+ };
+ return ret+" return "+compileE(body)+"; } })";
+ };
+ throw new Exception("Ill-formed lambda: "+fnc);
+ }
+
+ private static String compileEL ( Trees el ) throws Exception {
+ if (el.is_empty())
+ return "";
+ String ret = compileE(el.head());
+ for ( Tree a: el.tail() )
+ ret += ","+compileE(a);
+ return ret;
+ }
+
+ private static String compileE ( Tree e ) throws Exception {
+ if (e == null)
+ return "(new MR_byte(0))";
+ if (e.equals(#<true>))
+ return "(new MR_bool(true))";
+ else if (e.equals(#<false>))
+ return "(new MR_bool(false))";
+ else if (e.equals(#<null>))
+ return "(new MR_byte(0))";
+ else if (e.is_variable())
+ if (in_memory_repeat_vars.member(e))
+ return "Interpreter.lookup_global_binding(\""+e.toString()+"\")";
+ else return e.toString();
+ else if (e.is_long())
+ return "(new MR_int("+((LongLeaf)e).value()+"))";
+ else if (e.is_double())
+ return "(new MR_float("+((DoubleLeaf)e).value()+"))";
+ else if (e.is_string())
+ return "(new MR_string("+e.toString()+"))";
+ match e {
+ case callM(and,_,`x,`y):
+ return "(new MR_bool(((MR_bool)"+compileE(x)
+ +").get() && ((MR_bool)"+compileE(y)+").get()))";
+ case callM(or,_,`x,`y):
+ return "(new MR_bool(((MR_bool)"+compileE(x)
+ +").get() || ((MR_bool)"+compileE(y)+").get()))";
+ case callM(not,_,`x):
+ return "(new MR_bool(!((MR_bool)"+compileE(x)+").get()))";
+ case callM(`f,`n,...args):
+ if (!n.is_long())
+ fail;
+ String ret = "SystemFunctions."+ClassImporter.method_name((int)((LongLeaf)n).value())+"(";
+ Trees sig = ClassImporter.signature((int)((LongLeaf)n).value());
+ for (int i = 0; i < args.length(); i++)
+ ret += ((i > 0) ? ",(" : "(")+get_MR_type(sig.nth(i+1))+")("+compileE(args.nth(i))+")";
+ return ret+")";
+ case lambda(`v,`body):
+ return "new Lambda("+compilef(v.toString(),body)+")";
+ case nth(`x,`n):
+ return "(((Tuple)("+compileE(x)+")).get("+((LongLeaf)n).value()+"))";
+ case setNth(`x,`n,`v,`ret):
+ return "(((Tuple)("+compileE(x)+")).set("+((LongLeaf)n).value()+","+compileE(v)+","+compileE(ret)+"))";
+ case materialize(`u):
+ return "MapReduceAlgebra.materialize("+compileE(u)+")";
+ case let(`v,`u,`body):
+ return "(new Function () { public MRData eval ( final MRData "+v
+ +" ) { if ("+v+" instanceof Bag) ((Bag)"+v+").materialize(); return "
+ +compileE(body)+"; }; }).eval("+compileE(u)+")";
+ case cmap(`m,`s):
+ return "MapReduceAlgebra.cmap("+compileF(m)+",(Bag)("+compileE(s)+"))";
+ case filter(`p,`m,`s):
+ return "MapReduceAlgebra.filter("+compileF(p)+","+compileF(m)
+ +",(Bag)"+compileE(s)+")";
+ case map(`m,`s):
+ return "MapReduceAlgebra.map("+compileF(m)+",(Bag)"+compileE(s)+")";
+ case range(`min,`max):
+ return "MapReduceAlgebra.generator(((MR_long)"+compileE(min)+").get(),"
+ +"((MR_long)"+compileE(max)+").get())";
+ case call(`f,...args):
+ return "("+compileF(f)+".eval("+compileE(#<tuple(...args)>)+"))";
+ case tuple():
+ return "(new Tuple())";
+ case tuple(`x):
+ return "(new Tuple("+compileE(x)+"))";
+ case tuple(`a,...el):
+ String ret = "(new Tuple("+compileE(a);
+ for ( Tree x: el )
+ ret += ","+compileE(x);
+ return ret+"))";
+ case tagged_union(`n,`u):
+ return "(new Union((byte)"+((LongLeaf)n).value()+","+compileE(u)+"))";
+ case union_value(`x):
+ return "(((Union)"+compileE(x)+").value())";
+ case union_tag(`x):
+ return "(new MR_int(((Union)"+compileE(x)+").tag()))";
+ // used for shortcutting sync in bsp supersteps
+ case BAG():
+ return "SystemFunctions.bsp_empty_bag";
+ case TRUE():
+ return "SystemFunctions.bsp_true_value";
+ case FALSE():
+ return "SystemFunctions.bsp_false_value";
+ case `T():
+ if (is_collection(T))
+ return "(new Bag())";
+ else fail
+ case `T(e):
+ if (is_collection(T))
+ return "(new Bag("+compileE(e)+"))";
+ else fail
+ case `T(`a,...el):
+ if (!is_collection(T))
+ fail;
+ String ret = "(new Bag("+compileE(a);
+ for ( Tree x: el )
+ ret += ",(MRData)"+compileE(x);
+ return ret+"))";
+ case if(`c,`x,`y):
+ return "((((MR_bool)"+compileE(c)+").get())?"+compileE(x)+":"+compileE(y)+")";
+ case synchronize(`peer,`b):
+ return "SystemFunctions.synchronize(((MR_string)"+compileE(peer)+"),(MR_bool)"+compileE(b)+")";
+ case distribute(`peer,`s):
+ return "SystemFunctions.distribute(((MR_string)"+compileE(peer)+"),(Bag)"+compileE(s)+")";
+ case mapReduce(`mx,`my,`s,_):
+ return "MapReduceAlgebra.mapReduce("+compileF(mx)+","+compileF(my)+",(Bag)("+compileE(s)+"))";
+ case mapReduce2(`mx,`my,`r,`x,`y,_):
+ return "MapReduceAlgebra.mapReduce2("+compileF(mx)+","+compileF(my)+","+compileF(r)
+ +",(Bag)("+compileE(x)+"),(Bag)("+compileE(y)+"))";
+ case mapJoin(`kx,`ky,`r,`x,`y):
+ return "MapReduceAlgebra.mapJoin("+compileF(kx)+","+compileF(ky)+","+compileF(r)
+ +",(Bag)("+compileE(x)+"),(Bag)("+compileE(y)+"))";
+ case join(`kx,`ky,`r,`x,`y):
+ return "MapReduceAlgebra.join("+compileF(kx)+","+compileF(ky)+","+compileF(r)
+ +",(Bag)("+compileE(x)+"),(Bag)("+compileE(y)+"))";
+ case groupBy(`s):
+ return "MapReduceAlgebra.groupBy((Bag)("+compileE(s)+"))";
+ case index(`x,`n):
+ return "SystemFunctions.index((Bag)("+compileE(x)+"),"+compileE(n)+")";
+ case range(`x,`i,`j):
+ return "SystemFunctions.range((Bag)("+compileE(x)+"),"+compileE(i)+","+compileE(j)+")";
+ case map_index(`x,`key):
+ return "((Bag)("+compileE(x)+")).map_find("+compileE(key)+")";
+ case aggregate(`acc,`zero,`s):
+ return "MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)+",(Bag)("+compileE(s)+"))";
+ case Aggregate(`acc,`zero,`s):
+ return "MapReducePlan.aggregate("+compileF(acc)+","+compileE(zero)+","+compileM(s)+")";
+ case mergeGroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
+ return "MapReduceAlgebra.mergeGroupByJoin("+compileF(kx)+","+compileF(ky)
+ +","+compileF(gx)+","+compileF(gy)+","+compileF(m)+","+compileF(c)
+ +","+compileF(r)+",(Bag)"+compileE(x)+",(Bag)"+compileE(y)+")";
+ case function(tuple(...params),`tp,`body):
+ return compileF(e);
+ case typed(`x,_):
+ return compileE(x);
+ case apply(`f,tuple(...args)):
+ if (!f.is_variable())
+ fail;
+ match global_functions.lookup(f.toString()) {
+ case function(tuple(...params),`otp,`body):
+ String ret = "UserFunctions_"+user_functions_num+"."+f+"(";
+ if (args.is_empty())
+ return ret+")";
+ for ( int i = 0; i < params.length(); i++ )
+ match params.nth(i) {
+ case bind(_,`tp):
+ ret += ((i==0)?"":",")+"("+get_MR_type(tp)+")"+compileE(args.nth(i));
+ };
+ return ret+")";
+ };
+ case apply(`f,`arg):
+ if (!f.is_variable())
+ fail;
+ match global_functions.lookup(f.toString()) {
+ case function(tuple(...params),`otp,`body):
+ String ac = compileE(arg);
+ String ret = "UserFunctions_"+user_functions_num+"."+f+"(";
+ for ( int i = 0; i < params.length(); i++ )
+ match params.nth(i) {
+ case bind(_,`tp):
+ ret += ((i==0)?"":",")+"("+get_MR_type(tp)+")((Tuple)"+ac+").get("+i+")";
+ };
+ return ret+")";
+ };
+ case apply(`f,`arg):
+ if (!f.is_variable())
+ return "("+compileF(f)+").eval("+compileE(arg)+")";
+ else return "(((Lambda)"+compileE(f)+").lambda().eval("+compileE(arg)+"))";
+ case Collect(`s):
+ return "Plan.collect("+compileM(s)+")";
+ case trace(`x):
+ return compileE(x);
+ case _:
+ return compileM(e);
+ };
+ throw new Exception("Cannot compile: "+e);
+ }
+
+ final private static String compileM ( Tree e ) throws Exception {
+ match e {
+ case cMap(`f,`s):
+ return "MapReduceAlgebra.cmap("+compileF(f)+",(Bag)"+compileM(s)+")";
+ case AggregateMap(`f,`acc,`zero,`s):
+ return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)
+ +","+compileM(#<cMap(`f,`s)>)+"))";
+ case MapReduce(`m,`r,`s,_):
+ return "MapReduceAlgebra.mapReduce("+compileF(m)+","
+ +compileF(r)+",(Bag)"+compileM(s)+")";
+ case MapAggregateReduce(`m,`r,`acc,`zero,`s,_):
+ return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)
+ +","+compileE(zero)+","+compileM(#<MapReduce(`m,`r,`s)>)+"))";
+ case MapCombineReduce(`m,`c,`r,`s,_):
+ return "MapReduceAlgebra.mapReduce("+compileF(m)
+ +","+compileF(r)+",(Bag)"+compileM(s)+")";
+ case MapReduce2(`mx,`my,`c,`r,`x,`y,_):
+ return "MapReduceAlgebra.mapReduce2("+compileF(mx)+","+compileF(my)
+ +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
+ case MapReduce2(`mx,`my,`r,`x,`y,_):
+ return "MapReduceAlgebra.mapReduce2("+compileF(mx)+","+compileF(my)
+ +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
+ case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,_):
+ return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)
+ +","+compileM(#< MapReduce2(`mx,`my,`r,`x,`y)>)+"))";
+ case MapJoin(`kx,`ky,`r,`x,`y):
+ return "MapReduceAlgebra.mapJoin("+compileF(kx)+","+compileF(ky)
+ +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
+ case MapAggregateJoin(`kx,`ky,`r,`acc,`zero,`x,`y):
+ return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)
+ +","+compileE(zero)+","+compileM(#<MapJoin(`kx,`ky,`r,`x,`y)>)+"))";
+ case GroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
+ return "MapReduceAlgebra.groupByJoin("+compileF(kx)+","+compileF(ky)
+ +","+compileF(gx)+","+compileF(gy)+","+compileF(m)+","+compileF(c)
+ +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return "MapReduceAlgebra.crossProduct("+compileF(mx)+","+compileF(my)
+ +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
+ case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
+ return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)
+ +","+compileM(#<CrossProduct(`mx,`my,`r,`x,`y)>)+"))";
+ case BSPSource(`n,BinarySource(`file,_)):
+ if (Config.hadoop_mode)
+ return "Plan.binarySource("+compileE(n)+",((MR_string)"
+ +compileE(file)+").get())";
+ else return "(Bag)MapReduceAlgebra.read_binary("+compileE(n)+",((MR_string)"
+ +compileE(file)+").get())";
+ case BinarySource(`file,_):
+ if (Config.hadoop_mode)
+ return "Plan.binarySource(((MR_string)"+compileE(file)+").get())";
+ else return "(Bag)MapReduceAlgebra.read_binary(((MR_string)"+compileE(file)+").get())";
+ case BSPSource(`n,ParsedSource(`parser,`file,...args)):
+ if (!(n instanceof LongLeaf))
+ fail;
+ if (!Config.hadoop_mode)
+ return "MapReduceAlgebra.parsedSource(((MR_int)"+compileE(n)+").get(),\""
+ +parser+"\",((MR_string)"+compileE(file)+").get(),"
+ +reify(args)+")";
+ Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ return "Plan.parsedSource("+p.getName()+".class,((MR_string)"+compileE(file)+").get(),"
+ +reify(args)+")";
+ case ParsedSource(`parser,`file,...args):
+ if (!Config.hadoop_mode)
+ return "MapReduceAlgebra.parsedSource(\""+parser+"\",((MR_string)"
+ +compileE(file)+").get(),"+reify(args)+")";
+ Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
+ if (p == null)
+ throw new Error("Unknown parser: "+parser);
+ return "Plan.parsedSource("+p.getName()+".class,((MR_string)"+compileE(file)+").get(),"
+ +reify(args)+")";
+ case Merge(`x,`y):
+ return "((Bag)"+compileM(x)+").union((Bag)"+compileM(y)+")";
+ case Generator(`min,`max,`size):
+ return "MapReduceAlgebra.generator(((MR_long)"+compileE(min)+").get(),"
+ +"((MR_long)"+compileE(max)+").get())";
+ case BSP(`n,`superstep,`state,`o,...as):
+ String ds = "";
+ for ( Tree a: as )
+ ds += ",(Bag)("+compileM(a)+")";
+ return "MapReduceAlgebra.BSP("+((LongLeaf)n).value()+","
+ +compileF(superstep)+","+compileE(state)+","+o+","
+ +"new Bag[]{"+ds.substring(1)+"})";
+ case `v:
+ if (v.is_variable())
+ return v.toString();
+ };
+ throw new Exception("Cannot compile: "+e);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Config.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Config.java b/core/src/main/java/org/apache/mrql/Config.java
new file mode 100644
index 0000000..a957d89
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Config.java
@@ -0,0 +1,277 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.VariableLeaf;
+import java.util.ArrayList;
+import java.io.FileInputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+
+/** MRQL configuration parameters */
+final public class Config {
+ public static boolean loaded = false;
+
+ // true for using Hadoop HDFS file-system
+ public static boolean hadoop_mode = false;
+ // true for local execution (one node)
+ public static boolean local_mode = false;
+ // true for local execution (one node)
+ public static boolean distributed_mode = false;
+ // true for Hadoop map-reduce mode
+ public static boolean map_reduce_mode = false;
+ // true, for BSP mode using Hama
+ public static boolean bsp_mode = false;
+ // true, for Spark mode
+ public static boolean spark_mode = false;
+ // if true, it process the input interactively
+ public static boolean interactive = true;
+ // compile the MR functional arguments to Java bytecode at run-time
+ // (each task-tracker repeats the compilation at the MR setup time)
+ public static boolean compile_functional_arguments = true;
+ // if true, generates info about all compilation and optimization steps
+ public static boolean trace = false;
+ // number of worker nodes
+ public static int nodes = 2;
+ // true, to disable mapJoin
+ public static boolean noMapJoin = false;
+ // max distributed cache size for MapJoin (fragment-replicate join) in MBs
+ public static int mapjoin_size = 50;
+ // max entries for in-mapper combiner before they are flushed out
+ public static int map_cache_size = 100000;
+ // max number of bag elements to print
+ public static int max_bag_size_print = 20;
+ // max size of materialized vector before is spilled to a file:
+ public static int max_materialized_bag = 500000;
+ // max number of incoming messages before a sub-sync()
+ public static int bsp_msg_size = Integer.MAX_VALUE;
+ // number of elements per mapper to process the range min...max
+ public static long range_split_size = 100000;
+ // max number of streams to merge simultaneously
+ public static int max_merged_streams = 100;
+ // the directory for temporary files and spilled bags
+ public static String tmpDirectory = "/tmp/mrql_"+System.getProperty("user.name");
+ // true, if we want to derive a combine function for MapReduce
+ public static boolean use_combiner = true;
+ // true, if we can use the rule that fuses a groupBy with a join over the same key
+ public static boolean groupJoinOpt = true;
+ // true, if we can use the rule that converts a self-join into a simple mapreduce
+ public static boolean selfJoinOpt = true;
+ // true for run-time trace of plans
+ public static boolean trace_execution = false;
+ // true for extensive run-time trace of expressions & plans
+ public static boolean trace_exp_execution = false;
+ // true if you don't want to print statistics
+ public static boolean quiet_execution = false;
+ // true if this is during testing
+ public static boolean testing = false;
+ // true to display INFO log messages
+ public static boolean info = false;
+
+ /** store the configuration parameters */
+ public static void write ( Configuration conf ) {
+ conf.setBoolean("mrql.hadoop.mode",hadoop_mode);
+ conf.setBoolean("mrql.local.mode",local_mode);
+ conf.setBoolean("mrql.distributed.mode",distributed_mode);
+ conf.setBoolean("mrql.map.reduce.mode",map_reduce_mode);
+ conf.setBoolean("mrql.bsp.mode",bsp_mode);
+ conf.setBoolean("mrql.spark.mode",spark_mode);
+ conf.setBoolean("mrql.interactive",interactive);
+ conf.setBoolean("mrql.compile.functional.arguments",compile_functional_arguments);
+ conf.setBoolean("mrql.trace",trace);
+ conf.setInt("mrql.nodes",nodes);
+ conf.setInt("mrql.mapjoin.size",mapjoin_size);
+ conf.setInt("mrql.in.mapper.size",map_cache_size);
+ conf.setInt("mrql.max.bag.size.print",max_bag_size_print);
+ conf.setInt("mrql.max.materialized.bag",max_materialized_bag);
+ conf.setInt("mrql.bsp.msg.size",bsp_msg_size);
+ conf.setLong("mrql.range.split.size",range_split_size);
+ conf.setInt("mrql.max.merged.streams",max_merged_streams);
+ conf.set("mrql.tmp.directory",tmpDirectory);
+ conf.setBoolean("mrql.use.combiner",use_combiner);
+ conf.setBoolean("mrql.group.join.opt",groupJoinOpt);
+ conf.setBoolean("mrql.self.join.opt",selfJoinOpt);
+ conf.setBoolean("mrql.trace.execution",trace_execution);
+ conf.setBoolean("mrql.trace.exp.execution",trace_exp_execution);
+ conf.setBoolean("mrql.quiet.execution",quiet_execution);
+ conf.setBoolean("mrql.testing",testing);
+ conf.setBoolean("mrql.info",info);
+ }
+
+ /** load the configuration parameters */
+ public static void read ( Configuration conf ) {
+ if (loaded)
+ return;
+ loaded = true;
+ hadoop_mode = conf.getBoolean("mrql.hadoop.mode",hadoop_mode);
+ local_mode = conf.getBoolean("mrql.local.mode",local_mode);
+ distributed_mode = conf.getBoolean("mrql.distributed.mode",distributed_mode);
+ map_reduce_mode = conf.getBoolean("mrql.map.reduce.mode",map_reduce_mode);
+ bsp_mode = conf.getBoolean("mrql.bsp.mode",bsp_mode);
+ spark_mode = conf.getBoolean("mrql.spark.mode",spark_mode);
+ interactive = conf.getBoolean("mrql.interactive",interactive);
+ compile_functional_arguments = conf.getBoolean("mrql.compile.functional.arguments",compile_functional_arguments);
+ trace = conf.getBoolean("mrql.trace",trace);
+ nodes = conf.getInt("mrql.nodes",nodes);
+ mapjoin_size = conf.getInt("mrql.mapjoin.size",mapjoin_size);
+ map_cache_size = conf.getInt("mrql.in.mapper.size",map_cache_size);
+ max_bag_size_print = conf.getInt("mrql.max.bag.size.print",max_bag_size_print);
+ max_materialized_bag = conf.getInt("mrql.max.materialized.bag",max_materialized_bag);
+ bsp_msg_size = conf.getInt("mrql.bsp.msg.size",bsp_msg_size);
+ range_split_size = conf.getLong("mrql.range.split.size",range_split_size);
+ max_merged_streams = conf.getInt("mrql.max.merged.streams",max_merged_streams);
+ tmpDirectory = conf.get("mrql.tmp.directory");
+ use_combiner = conf.getBoolean("mrql.use.combiner",use_combiner);
+ groupJoinOpt = conf.getBoolean("mrql.group.join.opt",groupJoinOpt);
+ selfJoinOpt = conf.getBoolean("mrql.self.join.opt",selfJoinOpt);
+ trace_execution = conf.getBoolean("mrql.trace.execution",trace_execution);
+ trace_exp_execution = conf.getBoolean("mrql.trace.exp.execution",trace_exp_execution);
+ quiet_execution = conf.getBoolean("mrql.quiet.execution",quiet_execution);
+ testing = conf.getBoolean("mrql.testing",testing);
+ info = conf.getBoolean("mrql.info",info);
+ }
+
+ public static ArrayList<String> extra_args = new ArrayList<String>();
+
+ /** read configuration parameters from the Main args */
+ public static Bag parse_args ( String args[], Configuration conf ) throws Exception {
+ int i = 0;
+ int iargs = 0;
+ extra_args = new ArrayList<String>();
+ ClassImporter.load_classes();
+ interactive = true;
+ while (i < args.length) {
+ if (args[i].equals("-local")) {
+ local_mode = true;
+ i++;
+ } else if (args[i].equals("-dist")) {
+ distributed_mode = true;
+ i++;
+ } else if (args[i].equals("-reducers")) {
+ if (++i >= args.length)
+ throw new Error("Expected number of reductions");
+ nodes = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-bsp")) {
+ bsp_mode = true;
+ i++;
+ } else if (args[i].equals("-spark")) {
+ spark_mode = true;
+ i++;
+ } else if (args[i].equals("-bsp_tasks")) {
+ if (++i >= args.length && Integer.parseInt(args[i]) < 1)
+ throw new Error("Expected max number of bsp tasks > 1");
+ nodes = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-nodes")) {
+ if (++i >= args.length && Integer.parseInt(args[i]) < 1)
+ throw new Error("Expected number of nodes > 1");
+ nodes = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-bsp_msg_size")) {
+ if (++i >= args.length && Integer.parseInt(args[i]) < 10000)
+ throw new Error("Expected max number of bsp messages before subsync() > 10000");
+ bsp_msg_size = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-mapjoin_size")) {
+ if (++i >= args.length)
+ throw new Error("Expected number of MBs");
+ mapjoin_size = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-cache_size")) {
+ if (++i >= args.length)
+ throw new Error("Expected number of entries");
+ map_cache_size = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-tmp")) {
+ if (++i >= args.length)
+ throw new Error("Expected a temporary directory");
+ tmpDirectory = args[i];
+ i++;
+ } else if (args[i].equals("-bag_size")) {
+ if (++i >= args.length && Integer.parseInt(args[i]) < 10000)
+ throw new Error("Expected max size of materialized bag > 10000");
+ max_materialized_bag = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-bag_print")) {
+ if (++i >= args.length)
+ throw new Error("Expected number of bag elements to print");
+ max_bag_size_print = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-split_size")) {
+ if (++i >= args.length)
+ throw new Error("Expected a split size");
+ range_split_size = Long.parseLong(args[i]);
+ i++;
+ } else if (args[i].equals("-max_merged")) {
+ if (++i >= args.length)
+ throw new Error("Expected a max number of merged streams");
+ max_merged_streams = Integer.parseInt(args[i]);
+ i++;
+ } else if (args[i].equals("-trace")) {
+ trace = true;
+ i++;
+ } else if (args[i].equals("-C")) {
+ compile_functional_arguments = true;
+ i++;
+ } else if (args[i].equals("-NC")) {
+ compile_functional_arguments = false;
+ i++;
+ } else if (args[i].equals("-P")) {
+ trace_execution = true;
+ i++;
+ } else if (args[i].equals("-quiet")) {
+ quiet_execution = true;
+ i++;
+ } else if (args[i].equals("-info")) {
+ info = true;
+ i++;
+ } else if (args[i].equals("-trace_execution")) {
+ trace_execution = true;
+ trace_exp_execution = true;
+ compile_functional_arguments = false;
+ i++;
+ } else if (args[i].equals("-methods")) {
+ System.out.print("\nImported methods: ");
+ ClassImporter.print_methods();
+ System.out.println();
+ System.out.print("\nAggregations:");
+ Translator.print_aggregates();
+ System.out.println();
+ i++;
+ } else if (args[i].charAt(0) == '-')
+ throw new Error("Unknown MRQL parameter: "+args[i]);
+ else {
+ if (interactive) {
+ Main.query_file = args[i++];
+ interactive = false;
+ } else extra_args.add(args[i++]);
+ }
+ };
+ if (hadoop_mode)
+ write(conf);
+ Plan.conf = conf;
+ Bag b = new Bag();
+ for ( String s: extra_args )
+ b.add(new MR_string(s));
+ Interpreter.new_global_binding(new VariableLeaf("args").value(),b);
+ return b;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/DataSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/DataSet.java b/core/src/main/java/org/apache/mrql/DataSet.java
new file mode 100644
index 0000000..efe5646
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/DataSet.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.util.List;
+import java.util.ArrayList;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** The domain of the MRQL physical algebra is a set of DataSources */
+public class DataSet {
+ public ArrayList<DataSource> source; // multiple sources
+ public long counter; // a Hadoop user-defined counter used by the `repeat' operator
+ public long records; // total number of dataset records
+
+ /** Construct a DataSet that contains one DataSource
+ * @param s the given DataSource
+ * @param counter a Hadoop user-defined counter used by the `repeat' operator
+ * @param records total number of dataset records
+ */
+ DataSet ( DataSource s, long counter, long records ) {
+ source = new ArrayList<DataSource>();
+ source.add(s);
+ this.counter = counter;
+ this.records = records;
+ }
+
+ /** Construct a set of DataSources
+ * @param counter a Hadoop user-defined counter used by the `repeat' operator
+ * @param records total number of dataset records
+ */
+ DataSet ( long counter, long records ) {
+ source = new ArrayList<DataSource>();
+ this.counter = counter;
+ this.records = records;
+ }
+
+ /** add a DataSource to this DataSet */
+ public void add ( DataSource s ) {
+ source.add(s);
+ }
+
+ /** merge this DataSet with the given DataSet */
+ public void merge ( DataSet ds ) {
+ source.addAll(ds.source);
+ counter += ds.counter;
+ records += ds.records;
+ }
+
+ /** dataset size in bytes */
+ public long size ( Configuration conf ) {
+ long n = 0;
+ for (DataSource s: source)
+ n += s.size(conf);
+ return n;
+ }
+
+ /** return a single DataSource path by merging all the DataSource paths in this DataSet */
+ public String merge () {
+ Object[] ds = source.toArray();
+ String path = ((DataSource)ds[0]).path.toString();
+ for ( int i = 1; i < ds.length; i++ )
+ path += ","+((DataSource)ds[i]).path;
+ return path;
+ }
+
+ /** return the first num values */
+ public List<MRData> take ( int num ) {
+ int count = num;
+ ArrayList<MRData> res = new ArrayList<MRData>();
+ for ( DataSource s: source ) {
+ res.addAll(s.take(count));
+ if (res.size() < count)
+ count = count-res.size();
+ else return res;
+ };
+ return res;
+ }
+
+ /** accumulate all dataset values */
+ public MRData reduce ( MRData zero, Function acc ) {
+ MRData res = zero;
+ for ( DataSource s: source )
+ res = s.reduce(res,acc);
+ return res;
+ }
+
+ public String toString () {
+ String p = "<"+counter;
+ for (DataSource s: source)
+ p += ","+s;
+ return p+">";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/DataSource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/DataSource.java b/core/src/main/java/org/apache/mrql/DataSource.java
new file mode 100644
index 0000000..6a0ef4f
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/DataSource.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Iterator;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** A DataSource is any input data source, such as a text file, a key/value map, a data base, an intermediate file, etc */
+public class DataSource {
+ final public static String separator = "%%%";
+ public static DataSourceDirectory dataSourceDirectory = new DataSourceDirectory();
+ public static ParserDirectory parserDirectory = new ParserDirectory();
+ private static boolean loaded = false;
+
+ public String path;
+ public Class<? extends MRQLFileInputFormat> inputFormat;
+ public int source_num;
+ public boolean to_be_merged; // if the path is a directory with multiple files, merge them
+
+ final static class ParserDirectory extends HashMap<String,Class<? extends Parser>> {
+ }
+
+ /** A dictionary that maps data source paths to DataSource data.
+ * It assumes that each path can only be associated with a single data source format and parser
+ */
+ final static class DataSourceDirectory extends HashMap<String,DataSource> {
+ public void read ( Configuration conf ) {
+ clear();
+ for ( String s: conf.get("mrql.data.source.directory").split("@@@") ) {
+ String[] p = s.split("===");
+ put(p[0],DataSource.read(p[1],conf));
+ }
+ }
+
+ public String toString () {
+ String s = "";
+ for ( String k: keySet() )
+ s += "@@@"+k+"==="+get(k);
+ if (s.equals(""))
+ return s;
+ else return s.substring(3);
+ }
+
+ public DataSource get ( String name ) {
+ for ( Map.Entry<String,DataSource> e: entrySet() )
+ if (name.startsWith(e.getKey()))
+ return e.getValue();
+ return null;
+ }
+
+ public void distribute ( Configuration conf ) {
+ conf.set("mrql.data.source.directory",toString());
+ }
+ }
+
+ DataSource () {}
+
+ DataSource ( int source_num,
+ String path,
+ Class<? extends MRQLFileInputFormat> inputFormat,
+ Configuration conf ) {
+ this.source_num = source_num;
+ this.path = path;
+ this.inputFormat = inputFormat;
+ to_be_merged = false;
+ try {
+ Path p = new Path(path);
+ FileSystem fs = p.getFileSystem(conf);
+ String complete_path = fs.getFileStatus(p).getPath().toString();
+ //String complete_path = "file:"+path;
+ this.path = complete_path;
+ dataSourceDirectory.put(this.path,this);
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ public static void loadParsers() {
+ if (!loaded) {
+ DataSource.parserDirectory.put("xml",XMLParser.class);
+ DataSource.parserDirectory.put("json",JsonFormatParser.class);
+ DataSource.parserDirectory.put("line",LineParser.class);
+ loaded = true;
+ }
+ }
+
+ static {
+ loadParsers();
+ }
+
+ private static long size ( Path path, Configuration conf ) throws IOException {
+ FileStatus s = path.getFileSystem(conf).getFileStatus(path);
+ if (!s.isDir())
+ return s.getLen();
+ long size = 0;
+ for ( FileStatus fs: path.getFileSystem(conf).listStatus(path) )
+ size += fs.getLen();
+ return size;
+ }
+
+ /** data set size in bytes */
+ public long size ( Configuration conf ) {
+ try {
+ return size(new Path(path),conf);
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ public static DataSource read ( String buffer, Configuration conf ) {
+ try {
+ String[] s = buffer.split(separator);
+ int n = Integer.parseInt(s[1]);
+ if (s[0].equals("Binary"))
+ return new BinaryDataSource(n,s[2],conf);
+ else if (s[0].equals("Generator"))
+ return new GeneratorDataSource(n,s[2],conf);
+ else if (s[0].equals("Text"))
+ return new ParsedDataSource(n,s[3],parserDirectory.get(s[2]),((Node)Tree.parse(s[4])).children(),conf);
+ else throw new Error("Unrecognized data source: "+s[0]);
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ public static DataSource get ( String path, Configuration conf ) {
+ if (dataSourceDirectory.isEmpty())
+ dataSourceDirectory.read(conf);
+ return dataSourceDirectory.get(path);
+ }
+
+ public static DataSource getCached ( String remote_path, String local_path, Configuration conf ) {
+ DataSource ds = get(remote_path,conf);
+ ds.path = local_path;
+ dataSourceDirectory.put(local_path,ds);
+ return ds;
+ }
+
+ /** return the first num values */
+ public List<MRData> take ( int num ) {
+ int count = num;
+ try {
+ ArrayList<MRData> res = new ArrayList<MRData>();
+ Iterator<MRData> it = inputFormat.newInstance().materialize(new Path(path)).iterator();
+ for ( int i = num; (num < 0 || i > 0) && it.hasNext(); i-- )
+ if (Config.hadoop_mode && Config.bsp_mode)
+ res.add(((Tuple)it.next()).get(1)); // strip tag in BSP mode
+ else res.add(it.next());
+ return res;
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+
+ static Tuple tuple_container = new Tuple(new Tuple(),new Tuple());
+
+ /** accumulate all datasource values */
+ public MRData reduce ( MRData zero, final Function acc ) {
+ try {
+ MRData res = zero;
+ for ( MRData x: inputFormat.newInstance().materialize(new Path(path)) ) {
+ if (Config.hadoop_mode && Config.bsp_mode)
+ x = ((Tuple)x).get(1); // strip tag in BSP mode
+ tuple_container.set(0,res).set(1,x);
+ res = acc.eval(tuple_container);
+ };
+ return res;
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+}
[10/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Bag.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Bag.java b/src/main/java/core/Bag.java
deleted file mode 100644
index b092f30..0000000
--- a/src/main/java/core/Bag.java
+++ /dev/null
@@ -1,578 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.util.*;
-import java.io.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-
-
-/**
- * A sequence of MRData.
- * There are 3 kinds of Bag implementations, which are converted at run-time, when necessary:
- * 1) vector-based (materialized): used for small bags (when size is less than Config.max_materialized_bag);
- * 2) stream-based: can be traversed only once; implemented as Java iterators;
- * 3) spilled to a local file: can be accessed multiple times
- */
-public class Bag extends MRData implements Iterable<MRData> {
- private final static long serialVersionUID = 64629834894869L;
- enum Modes { STREAMED, MATERIALIZED, SPILLED };
- private transient Modes mode;
- private transient ArrayList<MRData> content; // content of a materialized bag
- private transient BagIterator iterator; // iterator for a streamed bag
- private transient boolean consumed; // true, if the stream has already been used
- private transient String path; // local path that contains the spilled bag
- private transient SequenceFile.Writer writer; // the file writer for spiled bags
-
- /**
- * create an empty bag as an ArrayList
- */
- public Bag () {
- mode = Modes.MATERIALIZED;
- content = new ArrayList<MRData>();
- }
-
- /**
- * create an empty bag as an ArrayList with a given capacity
- * @param size initial capacity
- */
- public Bag ( final int size ) {
- mode = Modes.MATERIALIZED;
- content = new ArrayList<MRData>(size);
- }
-
- /**
- * in-memory Bag construction (an ArrayList) initialized with data
- * @param as a vector of MRData to insert in the Bag
- */
- public Bag ( final MRData ...as ) {
- mode = Modes.MATERIALIZED;
- content = new ArrayList<MRData>(as.length);
- for ( MRData a: as )
- content.add(a);
- }
-
- /**
- * in-memory Bag construction (an ArrayList) initialized with data
- * @param as a vector of MRData to insert in the Bag
- */
- public Bag ( final List<MRData> as ) {
- mode = Modes.MATERIALIZED;
- content = new ArrayList<MRData>(as.size());
- for ( MRData a: as )
- content.add(a);
- }
-
- /**
- * lazy construction (stream-based) of a Bag
- * @param i the Iterator that generates the Bag elements
- */
- public Bag ( final BagIterator i ) {
- mode = Modes.STREAMED;
- iterator = i;
- consumed = false;
- }
-
- /** is the Bag stored in an ArrayList? */
- public boolean materialized () {
- return mode == Modes.MATERIALIZED;
- }
-
- /** is the Bag stream-based? */
- public boolean streamed () {
- return mode == Modes.STREAMED;
- }
-
- /** is the Bag spilled into a file? */
- public boolean spilled () {
- return mode == Modes.SPILLED;
- }
-
- /** return the Bag size (cache it in memory if necessary) */
- public int size () {
- if (materialized())
- return content.size();
- if (streamed() && consumed)
- throw new Error("*** The collection stream has already been consumed");
- int i = 0;
- for ( MRData e: this )
- i++;
- if (streamed())
- consumed = true;
- return i;
- }
-
- /** trim the ArrayList that caches the Bag */
- public void trim () {
- if (materialized())
- content.trimToSize();
- }
-
- /** get the n'th element of a Bag (cache it in memory if necessary)
- * @param n the index
- * @return the n'th element
- */
- public MRData get ( final int n ) {
- if (materialized())
- if (n < size())
- return content.get(n);
- else throw new Error("List index out of range: "+n);
- if (streamed() && consumed)
- throw new Error("*** The collection stream has already been consumed");
- int i = 0;
- for ( MRData e: this )
- if (i++ == n)
- return e;
- if (streamed())
- consumed = true;
- throw new Error("Cannot retrieve the "+n+"th element of a sequence");
- }
-
- /** replace the n'th element of a Bag with a new value
- * @param n the index
- * @param value the new value
- * @return the Bag
- */
- public Bag set ( final int n, final MRData value ) {
- if (!materialized())
- throw new Error("Cannot replace an element of a non-materialized sequence");
- content.set(n,value);
- return this;
- }
-
- /** add a new value to a Bag (cache it in memory if necessary)
- * @param x the new value
- */
- public void add ( final MRData x ) {
- materialize();
- if (!spilled() && Config.hadoop_mode
- && size() >= Config.max_materialized_bag)
- spill();
- if (spilled())
- try {
- if (writer == null) { // writer was closed earlier for reading
- FileSystem fs = FileSystem.getLocal(Plan.conf);
- writer = SequenceFile.createWriter(fs,Plan.conf,new Path(path),
- MRContainer.class,NullWritable.class,
- SequenceFile.CompressionType.NONE);
- System.err.println("*** Appending elements to a spilled Bag: "+path);
- };
- writer.append(new MRContainer(x),NullWritable.get());
- } catch (IOException e) {
- throw new Error("Cannot append an element to a spilled Bag: "+path);
- }
- else content.add(x);
- }
-
- /** add a new value to a Bag (cache it in memory if necessary)
- * @param x the new value
- * @return the Bag
- */
- public Bag add_element ( final MRData x ) {
- add(x);
- return this;
- }
-
- /** add the elements of a Bag to the end of this Bag
- * @param b the Bag whose elements are copied
- * @return the Bag
- */
- public Bag addAll ( final Bag b ) {
- for ( MRData e: b )
- add(e);
- return this;
- }
-
- /** make this Bag empty (cache it in memory if necessary) */
- public void clear () {
- if (materialized())
- content.clear();
- else if (streamed()) {
- if (writer != null)
- try {
- writer.close();
- } catch (IOException ex) {
- throw new Error(ex);
- };
- writer = null;
- path = null;
- mode = Modes.MATERIALIZED;
- content = new ArrayList<MRData>(100);
- };
- mode = Modes.MATERIALIZED;
- content = new ArrayList<MRData>();
- }
-
- /** cache the Bag to an ArrayList when is absolutely necessary */
- public void materialize () {
- if (materialized() || spilled())
- return;
- Iterator<MRData> iter = iterator();
- mode = Modes.MATERIALIZED;
- writer = null;
- path = null;
- content = new ArrayList<MRData>(100);
- while ( iter.hasNext() )
- add(iter.next());
- if (materialized()) // it may have been spilled
- content.trimToSize();
- iterator = null;
- }
-
- private static Random random_generator = new Random();
-
- private static String new_path ( FileSystem fs ) throws IOException {
- Path p;
- do {
- p = new Path("file://"+Config.tmpDirectory+"/mrql"+(random_generator.nextInt(1000000)));
- } while (p.getFileSystem(Plan.conf).exists(p));
- String path = p.toString();
- Plan.temporary_paths.add(path);
- return path;
- }
-
- /** spill the Bag to a local file */
- private void spill () {
- if (!spilled() && Config.hadoop_mode)
- try {
- if (Plan.conf == null)
- Plan.conf = Evaluator.evaluator.new_configuration();
- final FileSystem fs = FileSystem.getLocal(Plan.conf);
- path = new_path(fs);
- System.err.println("*** Spilling a Bag to a local file: "+path);
- final Path p = new Path(path);
- writer = SequenceFile.createWriter(fs,Plan.conf,new Path(path),
- MRContainer.class,NullWritable.class,
- SequenceFile.CompressionType.NONE);
- for ( MRData e: this )
- writer.append(new MRContainer(e),NullWritable.get());
- mode = Modes.SPILLED;
- content = null;
- iterator = null;
- } catch (Exception e) {
- throw new Error("Cannot spill a Bag to a local file");
- }
- }
-
- /**
- * sort the Bag (cache it in memory if necessary).
- * If the Bag was spilled during caching, use external sorting
- */
- public void sort () {
- materialize();
- if (spilled()) // if it was spilled during materialize()
- try { // use external sorting
- if (writer != null)
- writer.close();
- FileSystem fs = FileSystem.getLocal(Plan.conf);
- SequenceFile.Sorter sorter
- = new SequenceFile.Sorter(fs,new Plan.MRContainerKeyComparator(),
- MRContainer.class,NullWritable.class,Plan.conf);
- String out_path = new_path(fs);
- System.err.println("*** Using external sorting on a spilled bag "+path+" -> "+out_path);
- sorter.setMemory(64*1024*1024);
- sorter.sort(new Path(path),new Path(out_path));
- path = out_path;
- writer = null;
- } catch (Exception ex) {
- throw new Error("Cannot sort a spilled bag");
- }
- else Collections.sort(content);
- }
-
- /** return the Bag Iterator */
- public Iterator<MRData> iterator () {
- if (spilled())
- try {
- if (writer != null)
- writer.close();
- writer = null;
- return new BagIterator () {
- final FileSystem fs = FileSystem.getLocal(Plan.conf);
- final SequenceFile.Reader reader = new SequenceFile.Reader(fs,new Path(path),Plan.conf);
- final MRContainer key = new MRContainer();
- final NullWritable value = NullWritable.get();
- MRData data;
- public boolean hasNext () {
- try {
- if (!reader.next(key,value)) {
- reader.close();
- return false;
- };
- data = key.data();
- return true;
- } catch (IOException e) {
- throw new Error("Cannot collect values from a spilled Bag");
- }
- }
- public MRData next () {
- return data;
- }
- };
- } catch (IOException e) {
- throw new Error("Cannot collect values from a spilled Bag");
- }
- else if (materialized())
- return content.iterator();
- else {
- if (consumed) // this should never happen
- throw new Error("*** The collection stream has already been consumed");
- consumed = true;
- return iterator;
- }
- }
-
- /** cache MRData in memory by caching all Bags at any place and depth in MRData */
- public void materializeAll () {
- materialize();
- for (MRData e: this)
- e.materializeAll();
- }
-
- /** concatenate the elements of a given Bag to the elements of this Bag.
- * Does not change either Bag
- * @param s the given Bag
- * @return a new Bag
- */
- public Bag union ( final Bag s ) {
- final Iterator<MRData> i1 = iterator();
- final Iterator<MRData> i2 = s.iterator();
- return new Bag(new BagIterator () {
- boolean first = true;
- public boolean hasNext () {
- if (first)
- if (i1.hasNext())
- return true;
- else {
- first = false;
- return i2.hasNext();
- }
- else return i2.hasNext();
- }
- public MRData next () {
- if (first)
- return i1.next();
- else return i2.next();
- }
- });
- }
-
- /** does this Bag contain an element?
- * Cache this Bag in memory befor tetsing if necessary
- * @param x the element to find
- */
- public boolean contains ( final MRData x ) {
- if (materialized())
- return content.contains(x);
- if (streamed() && consumed)
- throw new Error("*** The collection stream has already been consumed");
- for ( MRData e: this )
- if (x.equals(e))
- return true;
- if (streamed())
- consumed = true;
- return false;
- }
-
- /** if this Bag is a Map from keys to values (a Bag of (key,value) pairs),
- * find the value with the given key; raise an error if not found
- * @param key the search key
- * @return the value associated with the key
- */
- public MRData map_find ( final MRData key ) {
- if (streamed() && consumed)
- throw new Error("*** The collection stream has already been consumed");
- for ( MRData e: this ) {
- Tuple p = (Tuple) e;
- if (key.equals(p.first()))
- return p.second();
- };
- if (streamed())
- consumed = true;
- throw new Error("key "+key+" not found in map");
- }
-
- /** if this Bag is a Map from keys to values (a Bag of (key,value) pairs),
- * does it contain a given key?
- * @param key the search key
- */
- public boolean map_contains ( final MRData key ) {
- if (streamed() && consumed)
- throw new Error("*** The collection stream has already been consumed");
- for ( MRData e: this )
- if (key.equals(((Tuple)e).first()))
- return true;
- if (streamed())
- consumed = true;
- return false;
- }
-
- /** the output serializer for Bag.
- * Stream-based Bags are serialized lazily (without having to cache the Bag in memory)
- */
- final public void write ( DataOutput out ) throws IOException {
- if (materialized()) {
- out.writeByte(MRContainer.BAG);
- WritableUtils.writeVInt(out,size());
- for ( MRData e: this )
- e.write(out);
- } else {
- out.writeByte(MRContainer.LAZY_BAG);
- for ( MRData e: this )
- e.write(out);
- out.writeByte(MRContainer.END_OF_LAZY_BAG);
- }
- }
-
- /** the input serializer for Bag */
- final public static Bag read ( DataInput in ) throws IOException {
- int n = WritableUtils.readVInt(in);
- Bag bag = new Bag(n);
- for ( int i = 0; i < n; i++ )
- bag.add(MRContainer.read(in));
- return bag;
- }
-
- /** a lazy input serializer for a Bag (it doesn't need to cache a Bag in memory) */
- public static Bag lazy_read ( final DataInput in ) throws IOException {
- Bag bag = new Bag(100);
- MRData data = MRContainer.read(in);
- while (data != MRContainer.end_of_lazy_bag) {
- bag.add(data);
- data = MRContainer.read(in);
- };
- if (bag.materialized())
- bag.content.trimToSize();
- return bag;
- }
-
- /** the input serializer for Bag */
- public void readFields ( DataInput in ) throws IOException {
- int n = WritableUtils.readVInt(in);
- mode = Modes.MATERIALIZED;
- iterator = null;
- path = null;
- writer = null;
- if (content == null)
- content = new ArrayList<MRData>(n);
- else {
- content.clear();
- content.ensureCapacity(n);
- };
- for ( int i = 0; i < n; i++ )
- add(MRContainer.read(in));
- }
-
- private void writeObject ( ObjectOutputStream out ) throws IOException {
- materialize();
- WritableUtils.writeVInt(out,size());
- for ( MRData e: this )
- e.write(out);
- }
-
- private void readObject ( ObjectInputStream in ) throws IOException, ClassNotFoundException {
- int n = WritableUtils.readVInt(in);
- mode = Modes.MATERIALIZED;
- iterator = null;
- path = null;
- writer = null;
- content = new ArrayList<MRData>(n);
- for ( int i = 0; i < n; i++ )
- add(MRContainer.read(in));
- }
-
- private void readObjectNoData () throws ObjectStreamException { };
-
- /** compare this Bag with a given Bag by comparing their associated elements */
- public int compareTo ( MRData x ) {
- Bag xt = (Bag)x;
- Iterator<MRData> xi = xt.iterator();
- Iterator<MRData> yi = iterator();
- while ( xi.hasNext() && yi.hasNext() ) {
- int c = xi.next().compareTo(yi.next());
- if (c < 0)
- return -1;
- else if (c > 0)
- return 1;
- };
- if (xi.hasNext())
- return -1;
- else if (yi.hasNext())
- return 1;
- else return 0;
- }
-
- /** compare this Bag with a given Bag by comparing their associated elements */
- final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
- try {
- int xn = WritableComparator.readVInt(x,xs);
- int xx = WritableUtils.decodeVIntSize(x[xs]);
- int yn = WritableComparator.readVInt(y,ys);
- int yy = WritableUtils.decodeVIntSize(y[ys]);
- for ( int i = 0; i < xn && i < yn; i++ ) {
- int k = MRContainer.compare(x,xs+xx,xl-xx,y,ys+yy,yl-yy,size);
- if (k != 0)
- return k;
- xx += size[0];
- yy += size[0];
- };
- size[0] = xx+1;
- if (xn > yn)
- return 1;
- if (xn < yn)
- return -1;
- return 0;
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- /** is this Bag equal to another Bag (order is important) */
- public boolean equals ( Object x ) {
- if (!(x instanceof Bag))
- return false;
- Bag xt = (Bag) x;
- Iterator<MRData> xi = xt.iterator();
- Iterator<MRData> yi = iterator();
- while ( xi.hasNext() && yi.hasNext() )
- if ( !xi.next().equals(yi.next()) )
- return false;
- return xi.hasNext() || yi.hasNext();
- }
-
- /** the hash code of this Bag is the XOR of the hash code of its elements */
- public int hashCode () {
- int h = 127;
- for ( MRData e: this )
- h ^= e.hashCode();
- return Math.abs(h);
- }
-
- /** show the first few Bag elements (controlled by -bag_print) */
- public String toString () {
- materialize();
- StringBuffer b = new StringBuffer("{ ");
- int i = 0;
- for ( MRData e: this )
- if ( i++ < Config.max_bag_size_print )
- b.append(((i>1)?", ":"")+e);
- else return b.append(", ... }").toString();
- return b.append(" }").toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/BagIterator.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/BagIterator.java b/src/main/java/core/BagIterator.java
deleted file mode 100644
index 0fd7354..0000000
--- a/src/main/java/core/BagIterator.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.util.Iterator;
-
-/** an Iterator over Bags */
-abstract public class BagIterator implements Iterator<MRData> {
- public void remove () {
- throw new Error("Bag deletions are not permitted");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/BinaryDataSource.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/BinaryDataSource.java b/src/main/java/core/BinaryDataSource.java
deleted file mode 100644
index d4338ec..0000000
--- a/src/main/java/core/BinaryDataSource.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.hadoop.conf.Configuration;
-
-
-/** A DataSource used for storing intermediate results and data dumps */
-final public class BinaryDataSource extends DataSource {
- BinaryDataSource ( int source_num, String path, Configuration conf ) {
- super(source_num,path,Evaluator.evaluator.binaryInputFormat(),conf);
- }
-
- BinaryDataSource ( String path, Configuration conf ) {
- super(-1,path,Evaluator.evaluator.binaryInputFormat(),conf);
- }
-
- public String toString () {
- return "Binary"+separator+source_num+separator+path;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/ClassImporter.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/ClassImporter.java b/src/main/java/core/ClassImporter.java
deleted file mode 100644
index 8efc1ae..0000000
--- a/src/main/java/core/ClassImporter.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.lang.reflect.Method;
-import org.apache.mrql.gen.*;
-import java.util.*;
-
-/** imports external Java methods into MRQL */
-final public class ClassImporter {
- final static boolean trace_imported_methods = false;
-
- final static String[] object_methods
- = { "hashCode", "getClass", "wait", "equals", "toString", "notify", "notifyAll" };
-
- static Vector<MethodInfo> methods = new Vector<MethodInfo>();
-
- public static void load_classes () {
- if (methods == null)
- methods = new Vector<MethodInfo>();
- if (methods.size() == 0) {
- importClass("org.apache.mrql.SystemFunctions");
- //****** import your classes with user-defined functions here
- }
- }
-
- private static boolean object_method ( String s ) {
- for (int i = 0; i < object_methods.length; i++)
- if (object_methods[i].equals(s))
- return true;
- return false;
- }
-
- private static Tree getType ( Class<?> c ) {
- String cn = c.getCanonicalName();
- Class<?>[] inf = c.getInterfaces();
- if (cn.equals("org.apache.mrql.MRData"))
- return new VariableLeaf("any");
- if (cn.startsWith("org.apache.mrql.MR_"))
- return new VariableLeaf(cn.substring(19));
- if (cn.equals("org.apache.mrql.Bag"))
- return new Node("bag",new Trees(new VariableLeaf("any")));
- if (cn.equals("org.apache.mrql.Inv"))
- return new VariableLeaf("any");
- if (cn.equals("org.apache.mrql.Union"))
- return new VariableLeaf("union");
- if (cn.equals("org.apache.mrql.Lambda"))
- return new VariableLeaf("any");
- if (inf.length > 0 && inf[0].equals("org.apache.mrql.MRData"))
- return new VariableLeaf("any");
- throw new Error("Unsupported type in imported method: "+cn);
- }
-
- private static Trees signature ( Method m ) {
- Class<?> co = m.getReturnType();
- Class<?>[] cs = m.getParameterTypes();
- Trees as = new Trees(getType(co));
- for (int i = 0; i < cs.length; i++)
- as = as.append(getType(cs[i]));
- return as;
- }
-
- public static String method_name ( int method_number ) {
- return methods.get(method_number).name;
- }
-
- public static Trees signature ( int method_number ) {
- return methods.get(method_number).signature;
- }
-
- /** import all Java methods from a given Java class */
- public static void importClass ( String class_name ) {
- try {
- Method[] ms = Class.forName(class_name).getMethods();
- Vector<MethodInfo> mv = new Vector<MethodInfo>();
- for (int i = 0; i < ms.length; i++)
- if (!object_method(ms[i].getName()) && ms[i].getModifiers() == 9)
- try {
- Trees sig = signature(ms[i]);
- MethodInfo m = new MethodInfo(ms[i].getName(),sig,ms[i]);
- mv.add(m);
- methods.add(m);
- } catch ( Exception e ) {
- System.out.println("Warning: method "+ms[i].getName()+" cannot be imported");
- System.out.println(e);
- throw new Error("");
- };
- Collections.sort(methods);
- if (Translator.functions == null)
- Translator.functions = Trees.nil;
- for ( MethodInfo m: methods )
- Translator.functions = Translator.functions.append(new Node(m.name,m.signature));
- if (trace_imported_methods) {
- System.out.print("Importing methods: ");
- for (int i = 0; i < mv.size(); i++ )
- System.out.print(mv.get(i).name+mv.get(i).signature.tail()
- +":"+mv.get(i).signature.head()+" ");
- System.out.println();
- }
- } catch (ClassNotFoundException x) {
- throw new Error("Undefined class: "+class_name);
- }
- }
-
- /** import a Java method with a given name from a given Java class */
- public static void importMethod ( String class_name, String method_name ) {
- try {
- Method[] ms = Class.forName(class_name).getMethods();
- MethodInfo m = null;
- for (int i = 0; i < ms.length; i++)
- if (ms[i].getName().equals(method_name)
- && !object_method(ms[i].getName()) && ms[i].getModifiers() == 9) {
- Trees sig = signature(ms[i]);
- m = new MethodInfo(ms[i].getName(),sig,ms[i]);
- Translator.functions = Translator.functions.append(new Node(ms[i].getName(),sig));
- break;
- };
- if (m == null)
- throw new Error("No such method: "+method_name);
- methods.add(m);
- Collections.sort(methods);
- if (trace_imported_methods)
- System.out.println("Importing method: "+m.name+m.signature.tail()
- +":"+m.signature.head()+" ");
- } catch (ClassNotFoundException x) {
- throw new Error("Undefined class: "+class_name);
- }
- }
-
- public static void print_methods () {
- for (int i = 0; i < methods.size(); i++ ) {
- MethodInfo m = methods.get(i);
- System.out.print(" "+m.name+":"+m.signature.tail()+"->"+m.signature.head());
- };
- }
-
- /** return the method specification of a system method with a given name over some expressions;
- * When the method is overloaded, find the most specific (in terms of arg subtyping)
- * @param method_name the given method name
- * @param args the method expressions
- * @return the method specification
- */
- public static Tree find_method ( String method_name, Trees args ) {
- for (int i = 0; i < methods.size(); i++ ) {
- MethodInfo m = methods.get(i);
- if (m.name.equals(method_name) && TypeInference.subtype(args,m.signature.tail()))
- return m.signature.head();
- };
- return null;
- }
-
- /** return the method number of a system method with a given name over some expressions;
- * When the method is overloaded, find the most specific (in terms of arg subtyping)
- * @param method_name the given method name
- * @param args the method expressions
- * @return the method number
- */
- public static int find_method_number ( String method_name, Trees args ) {
- for (int i = 0; i < methods.size(); i++ ) {
- MethodInfo m = methods.get(i);
- if (m.name.equals(method_name) && TypeInference.subtype(args,m.signature.tail()))
- return i;
- };
- return -1;
- }
-
- /** call a system method with a given number over MRData
- * @param method_number the method number
- * @param args in input arguments
- * @return the result of invoking this method over the args
- */
- public static MRData call ( int method_number, MRData... args ) {
- if (method_number < 0 || method_number >= methods.size())
- throw new Error("Run-time error (unknown method name)");
- MethodInfo m = methods.get(method_number);
- try {
- return (MRData)m.method.invoke(null,(Object[])args);
- } catch (Exception e) {
- Tuple t = new Tuple(args.length);
- for ( int i = 0; i < args.length; i++ )
- t.set(i,args[i]);
- System.err.println("Run-time error in method call: "+m.name+t+" of type "
- +m.signature.tail()+"->"+m.signature.head());
- throw new Error(e.toString());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Compiler.gen
----------------------------------------------------------------------
diff --git a/src/main/java/core/Compiler.gen b/src/main/java/core/Compiler.gen
deleted file mode 100644
index 668cdc8..0000000
--- a/src/main/java/core/Compiler.gen
+++ /dev/null
@@ -1,584 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.*;
-import javax.tools.*;
-import javax.tools.JavaCompiler.CompilationTask;
-import java.lang.reflect.*;
-import java.util.*;
-import java.net.*;
-import java.util.jar.*;
-import org.apache.hadoop.io.WritableComparable;
-
-
-/** compilation of MRQL expressions to Java code and then to Java bytecode */
-final public class Compiler extends Translator {
- static JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
- static DiagnosticCollector<JavaFileObject> diagnostics = new DiagnosticCollector<JavaFileObject>();
- final static String tmp_dir = "/tmp/mrql_jar_"+System.getProperty("user.name");
- public static String jar_path;
- static Trees in_memory_repeat_vars = #[];
- static int lambda_num = 0;
- static int user_functions_num = 0;
-
- /** Compile the MRQL functional arguments into Java bytecode */
- final private static class JavaSourceFromString extends SimpleJavaFileObject {
- final String code;
-
- JavaSourceFromString ( String name, String code ) {
- super(URI.create("string:///org/apache/mrql/" + name.replace('.','/') + Kind.SOURCE.extension),Kind.SOURCE);
- this.code = code;
- }
-
- @Override
- public CharSequence getCharContent ( boolean ignoreEncodingErrors ) {
- return code;
- }
- }
-
- private static String new_lambda_name () {
- return "MRQL_Lambda_"+(lambda_num++);
- }
-
- private static void add2jar ( File source, int offset, JarOutputStream target ) throws IOException {
- if (source.isDirectory()) {
- String name = source.getPath();
- if (name.length() > offset) {
- JarEntry entry = new JarEntry(name.substring(offset));
- entry.setTime(source.lastModified());
- target.putNextEntry(entry);
- target.closeEntry();
- };
- for ( File nestedFile: source.listFiles() )
- add2jar(nestedFile,offset,target);
- } else {
- JarEntry entry = new JarEntry(source.getPath().substring(offset));
- entry.setTime(source.lastModified());
- target.putNextEntry(entry);
- BufferedInputStream in = new BufferedInputStream(new FileInputStream(source));
- byte[] buffer = new byte[1024];
- int count = 1;
- while (count > 0) {
- count = in.read(buffer);
- if (count > 0)
- target.write(buffer,0,count);
- };
- target.closeEntry();
- in.close();
- }
- }
-
- private static void remove ( File file ) throws IOException {
- if (file.isDirectory())
- for ( File nestedFile: file.listFiles() )
- remove(nestedFile);
- file.delete();
- }
-
- public static void clean () throws IOException {
- in_memory_repeat_vars = #[];
- remove(new File(tmp_dir));
- }
-
- final private static Tree compile ( Tree e, StringBuffer out ) throws Exception {
- match e {
- case repeat(lambda(`v,`body),`s,`n):
- in_memory_repeat_vars = in_memory_repeat_vars.cons(v);
- return #<repeat(lambda(`v,`(compile(body,out))),`(compile(s,out)),`n)>;
- case `f(...al):
- if (!plans_with_distributed_lambdas.member(#<`f>))
- fail;
- Trees nl = #[];
- for ( Tree a: al)
- match Interpreter.closure(a,Interpreter.global_env) {
- case lambda(`v,`body):
- String fname = new_lambda_name();
- StringBuffer sb = new StringBuffer(1000);
- sb.append("final class "+fname+" extends Function {\n");
- sb.append(fname+" () {}\n");
- sb.append("final public MRData eval ( final MRData "+v
- +" ) { return "+compileE(body)+"; }\n}\n");
- out.append(sb);
- nl = nl.append(#<compiled(`fname,`a)>);
- case _: nl = nl.append(compile(a,out));
- };
- return #<`f(...nl)>;
- case `f(...al):
- Trees nl = #[];
- for ( Tree a: al)
- nl = nl.append(compile(a,out));
- return #<`f(...nl)>;
- };
- return e;
- }
-
- private static StringBuffer out;
-
- /** compile the functional arguments of the MRQL operators using the Java compiler
- * @param query the expression to compile
- * @return the query with all functional arguments compiled to Java bytecode
- */
- final public static Tree compile ( Tree query ) {
- try {
- user_functions_num = lambda_num++;
- // remove the old jar
- if (jar_path != null)
- remove(new File(jar_path));
- jar_path = tmp_dir+"/mrql_args_"+(new Random().nextInt(1000000))+".jar";
- out = new StringBuffer(1000);
- out.append("package org.apache.mrql;\n");
- Tree nq = compile(query,out);
- StringBuffer sb = new StringBuffer(1000);
- for ( String f: global_functions )
- match global_functions.lookup(f) {
- case function(tuple(...params),`otp,`body):
- sb.append("final public static "+get_MR_type(otp)+" "+f);
- if (params.is_empty())
- sb.append(" ()");
- else {
- match params.head() {
- case bind(`v,`tp):
- sb.append(" ( final "+get_MR_type(tp)+" "+v);
- };
- for ( Tree var: params.tail() )
- match var {
- case bind(`v,`tp):
- sb.append(", final "+get_MR_type(tp)+" "+v);
- }
- sb.append(" ) { return ("+get_MR_type(otp)+")");
- sb.append(compileE(body));
- sb.append("; }\n");
- }
- };
- out.append("final class UserFunctions_"+user_functions_num+" {\n");
- out.append(sb);
- out.append("}\n");
- String code = out.toString();
- //System.out.println(code);
- JavaFileObject file = new JavaSourceFromString("UserFunctions_"+user_functions_num,code);
- Iterable<? extends JavaFileObject> compilationUnits = Arrays.asList(file);
- List<String> optionList = new ArrayList<String>();
- (new File(tmp_dir)).mkdir();
- String dir = tmp_dir+"/classes_"+(new Random().nextInt(1000000));
- File fdir = new File(dir);
- fdir.mkdir();
- ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
- String classpath = jar_path;
- String separator = System.getProperty("path.separator");
- for ( URL url: ((URLClassLoader) classLoader).getURLs() )
- classpath += separator+url.getFile();
- // use hadoop core jar
- classpath += separator + WritableComparable.class.getProtectionDomain().getCodeSource().getLocation().toString();
- optionList.addAll(Arrays.asList("-classpath",classpath));
- optionList.addAll(Arrays.asList("-d",dir));
- CompilationTask task = compiler.getTask(null,null,diagnostics,optionList,null,compilationUnits);
- boolean success = task.call();
- if (!success)
- for ( Diagnostic d: diagnostics.getDiagnostics() )
- System.err.println("*** Compilation error at line "+d.getLineNumber()+" position "
- +d.getColumnNumber()+": "+d.getMessage(Locale.US));
- Manifest manifest = new Manifest();
- manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION,"1.0");
- JarOutputStream target = new JarOutputStream(new FileOutputStream(jar_path),manifest);
- add2jar(new File(dir+"/"),dir.length()+1,target);
- target.close();
- remove(fdir);
- return nq;
- } catch (Exception e) {
- System.err.println("*** Warning: Unable to compile the query:\n"+query);
- if (Config.trace)
- e.printStackTrace(System.err);
- return query;
- }
- }
-
- /** load the Java class of the anonymous function with name lambda_name */
- final public static Function compiled ( ClassLoader cl, String lambda_name ) throws Exception {
- URL[] urls = ((URLClassLoader) cl).getURLs();
- URL[] new_urls = new URL[urls.length+1];
- for ( int i = 0; i < urls.length; i++ )
- new_urls[i+1] = urls[i];
- new_urls[0] = new URL("file://"+jar_path);
- URLClassLoader loader = new URLClassLoader(new_urls,cl);
- Class c = loader.loadClass("org.apache.mrql."+lambda_name);
- Constructor cc = c.getDeclaredConstructors()[0];
- cc.setAccessible(true);
- return (Function)cc.newInstance();
- }
-
- /** The Java type of an MRQL type */
- private static String get_MR_type ( Tree type ) {
- match type {
- case boolean: return "MR_bool";
- case byte: return "MR_byte";
- case short: return "MR_short";
- case int: return "MR_int";
- case long: return "MR_long";
- case float: return "MR_float";
- case double: return "MR_double";
- case char: return "MR_char";
- case string: return "MR_string";
- case union: return "Union";
- case bag(...): return "Bag";
- case list(...): return "Bag";
- };
- return "MRData";
- }
-
- private static Trees remove_duplicates ( Trees al ) {
- if (al.is_empty())
- return al;
- Trees el = remove_duplicates(al.tail());
- if (el.member(al.head()))
- return el;
- else return el.cons(al.head());
- }
-
- /** lambda lifting: generate Java code from an anonymous function */
- private static String compilef ( String v, Tree body ) throws Exception {
- String fname = new_lambda_name();
- Trees free_vars = remove_duplicates(free_variables(body,#[`v]));
- StringBuffer sb = new StringBuffer(1000);
- sb.append("final class "+fname+" extends Function {\n");
- for ( Tree var: free_vars )
- sb.append("MRData "+var+"; ");
- sb.append("\npublic "+fname+" (");
- if (free_vars.is_empty())
- sb.append(") {}\n");
- else {
- sb.append(" MRData "+free_vars.head());
- for ( Tree var: free_vars.tail() )
- sb.append(", MRData "+var);
- sb.append(" ) { ");
- for ( Tree var: free_vars )
- sb.append("this."+var+" = "+var+"; ");
- sb.append("}\n");
- };
- sb.append("final public MRData eval ( final MRData "+v
- +" ) { return "+compileE(body)+"; }\n}\n");
- out.append(sb);
- String s = "new "+fname+"(";
- if (!free_vars.is_empty()) {
- s += free_vars.head();
- for ( Tree var: free_vars.tail() )
- s += ","+var;
- };
- return s+")";
- }
-
- private static String compileF ( Tree fnc ) throws Exception {
- match fnc {
- case lambda(`v,`b):
- return compilef(v.toString(),b);
- case compiled(`f,`lm):
- // recompile the function
- String s = compileF(lm);
- ((Node)fnc).children().head = new VariableLeaf(s.substring(4,s.indexOf("("))); //destructive
- return s;
- case function(tuple(...params),`tp,`body):
- String ret = "new Lambda(new Function () { "
- +"final public MRData eval ( final MRData _x ) { ";
- for ( int i = 0; i < params.length(); i++ )
- match params.nth(i) {
- case bind(`v,_):
- ret += "final MRData "+v+" = ((Tuple)_x).get("+i+"); ";
- };
- return ret+" return "+compileE(body)+"; } })";
- };
- throw new Exception("Ill-formed lambda: "+fnc);
- }
-
- private static String compileEL ( Trees el ) throws Exception {
- if (el.is_empty())
- return "";
- String ret = compileE(el.head());
- for ( Tree a: el.tail() )
- ret += ","+compileE(a);
- return ret;
- }
-
- private static String compileE ( Tree e ) throws Exception {
- if (e == null)
- return "(new MR_byte(0))";
- if (e.equals(#<true>))
- return "(new MR_bool(true))";
- else if (e.equals(#<false>))
- return "(new MR_bool(false))";
- else if (e.equals(#<null>))
- return "(new MR_byte(0))";
- else if (e.is_variable())
- if (in_memory_repeat_vars.member(e))
- return "Interpreter.lookup_global_binding(\""+e.toString()+"\")";
- else return e.toString();
- else if (e.is_long())
- return "(new MR_int("+((LongLeaf)e).value()+"))";
- else if (e.is_double())
- return "(new MR_float("+((DoubleLeaf)e).value()+"))";
- else if (e.is_string())
- return "(new MR_string("+e.toString()+"))";
- match e {
- case callM(and,_,`x,`y):
- return "(new MR_bool(((MR_bool)"+compileE(x)
- +").get() && ((MR_bool)"+compileE(y)+").get()))";
- case callM(or,_,`x,`y):
- return "(new MR_bool(((MR_bool)"+compileE(x)
- +").get() || ((MR_bool)"+compileE(y)+").get()))";
- case callM(not,_,`x):
- return "(new MR_bool(!((MR_bool)"+compileE(x)+").get()))";
- case callM(`f,`n,...args):
- if (!n.is_long())
- fail;
- String ret = "SystemFunctions."+ClassImporter.method_name((int)((LongLeaf)n).value())+"(";
- Trees sig = ClassImporter.signature((int)((LongLeaf)n).value());
- for (int i = 0; i < args.length(); i++)
- ret += ((i > 0) ? ",(" : "(")+get_MR_type(sig.nth(i+1))+")("+compileE(args.nth(i))+")";
- return ret+")";
- case lambda(`v,`body):
- return "new Lambda("+compilef(v.toString(),body)+")";
- case nth(`x,`n):
- return "(((Tuple)("+compileE(x)+")).get("+((LongLeaf)n).value()+"))";
- case setNth(`x,`n,`v,`ret):
- return "(((Tuple)("+compileE(x)+")).set("+((LongLeaf)n).value()+","+compileE(v)+","+compileE(ret)+"))";
- case materialize(`u):
- return "MapReduceAlgebra.materialize("+compileE(u)+")";
- case let(`v,`u,`body):
- return "(new Function () { public MRData eval ( final MRData "+v
- +" ) { if ("+v+" instanceof Bag) ((Bag)"+v+").materialize(); return "
- +compileE(body)+"; }; }).eval("+compileE(u)+")";
- case cmap(`m,`s):
- return "MapReduceAlgebra.cmap("+compileF(m)+",(Bag)("+compileE(s)+"))";
- case filter(`p,`m,`s):
- return "MapReduceAlgebra.filter("+compileF(p)+","+compileF(m)
- +",(Bag)"+compileE(s)+")";
- case map(`m,`s):
- return "MapReduceAlgebra.map("+compileF(m)+",(Bag)"+compileE(s)+")";
- case range(`min,`max):
- return "MapReduceAlgebra.generator(((MR_long)"+compileE(min)+").get(),"
- +"((MR_long)"+compileE(max)+").get())";
- case call(`f,...args):
- return "("+compileF(f)+".eval("+compileE(#<tuple(...args)>)+"))";
- case tuple():
- return "(new Tuple())";
- case tuple(`x):
- return "(new Tuple("+compileE(x)+"))";
- case tuple(`a,...el):
- String ret = "(new Tuple("+compileE(a);
- for ( Tree x: el )
- ret += ","+compileE(x);
- return ret+"))";
- case tagged_union(`n,`u):
- return "(new Union((byte)"+((LongLeaf)n).value()+","+compileE(u)+"))";
- case union_value(`x):
- return "(((Union)"+compileE(x)+").value())";
- case union_tag(`x):
- return "(new MR_int(((Union)"+compileE(x)+").tag()))";
- // used for shortcutting sync in bsp supersteps
- case BAG():
- return "SystemFunctions.bsp_empty_bag";
- case TRUE():
- return "SystemFunctions.bsp_true_value";
- case FALSE():
- return "SystemFunctions.bsp_false_value";
- case `T():
- if (is_collection(T))
- return "(new Bag())";
- else fail
- case `T(e):
- if (is_collection(T))
- return "(new Bag("+compileE(e)+"))";
- else fail
- case `T(`a,...el):
- if (!is_collection(T))
- fail;
- String ret = "(new Bag("+compileE(a);
- for ( Tree x: el )
- ret += ",(MRData)"+compileE(x);
- return ret+"))";
- case if(`c,`x,`y):
- return "((((MR_bool)"+compileE(c)+").get())?"+compileE(x)+":"+compileE(y)+")";
- case synchronize(`peer,`b):
- return "SystemFunctions.synchronize(((MR_string)"+compileE(peer)+"),(MR_bool)"+compileE(b)+")";
- case distribute(`peer,`s):
- return "SystemFunctions.distribute(((MR_string)"+compileE(peer)+"),(Bag)"+compileE(s)+")";
- case mapReduce(`mx,`my,`s,_):
- return "MapReduceAlgebra.mapReduce("+compileF(mx)+","+compileF(my)+",(Bag)("+compileE(s)+"))";
- case mapReduce2(`mx,`my,`r,`x,`y,_):
- return "MapReduceAlgebra.mapReduce2("+compileF(mx)+","+compileF(my)+","+compileF(r)
- +",(Bag)("+compileE(x)+"),(Bag)("+compileE(y)+"))";
- case mapJoin(`kx,`ky,`r,`x,`y):
- return "MapReduceAlgebra.mapJoin("+compileF(kx)+","+compileF(ky)+","+compileF(r)
- +",(Bag)("+compileE(x)+"),(Bag)("+compileE(y)+"))";
- case join(`kx,`ky,`r,`x,`y):
- return "MapReduceAlgebra.join("+compileF(kx)+","+compileF(ky)+","+compileF(r)
- +",(Bag)("+compileE(x)+"),(Bag)("+compileE(y)+"))";
- case groupBy(`s):
- return "MapReduceAlgebra.groupBy((Bag)("+compileE(s)+"))";
- case index(`x,`n):
- return "SystemFunctions.index((Bag)("+compileE(x)+"),"+compileE(n)+")";
- case range(`x,`i,`j):
- return "SystemFunctions.range((Bag)("+compileE(x)+"),"+compileE(i)+","+compileE(j)+")";
- case map_index(`x,`key):
- return "((Bag)("+compileE(x)+")).map_find("+compileE(key)+")";
- case aggregate(`acc,`zero,`s):
- return "MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)+",(Bag)("+compileE(s)+"))";
- case Aggregate(`acc,`zero,`s):
- return "MapReducePlan.aggregate("+compileF(acc)+","+compileE(zero)+","+compileM(s)+")";
- case mergeGroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
- return "MapReduceAlgebra.mergeGroupByJoin("+compileF(kx)+","+compileF(ky)
- +","+compileF(gx)+","+compileF(gy)+","+compileF(m)+","+compileF(c)
- +","+compileF(r)+",(Bag)"+compileE(x)+",(Bag)"+compileE(y)+")";
- case function(tuple(...params),`tp,`body):
- return compileF(e);
- case typed(`x,_):
- return compileE(x);
- case apply(`f,tuple(...args)):
- if (!f.is_variable())
- fail;
- match global_functions.lookup(f.toString()) {
- case function(tuple(...params),`otp,`body):
- String ret = "UserFunctions_"+user_functions_num+"."+f+"(";
- if (args.is_empty())
- return ret+")";
- for ( int i = 0; i < params.length(); i++ )
- match params.nth(i) {
- case bind(_,`tp):
- ret += ((i==0)?"":",")+"("+get_MR_type(tp)+")"+compileE(args.nth(i));
- };
- return ret+")";
- };
- case apply(`f,`arg):
- if (!f.is_variable())
- fail;
- match global_functions.lookup(f.toString()) {
- case function(tuple(...params),`otp,`body):
- String ac = compileE(arg);
- String ret = "UserFunctions_"+user_functions_num+"."+f+"(";
- for ( int i = 0; i < params.length(); i++ )
- match params.nth(i) {
- case bind(_,`tp):
- ret += ((i==0)?"":",")+"("+get_MR_type(tp)+")((Tuple)"+ac+").get("+i+")";
- };
- return ret+")";
- };
- case apply(`f,`arg):
- if (!f.is_variable())
- return "("+compileF(f)+").eval("+compileE(arg)+")";
- else return "(((Lambda)"+compileE(f)+").lambda().eval("+compileE(arg)+"))";
- case Collect(`s):
- return "Plan.collect("+compileM(s)+")";
- case trace(`x):
- return compileE(x);
- case _:
- return compileM(e);
- };
- throw new Exception("Cannot compile: "+e);
- }
-
- final private static String compileM ( Tree e ) throws Exception {
- match e {
- case cMap(`f,`s):
- return "MapReduceAlgebra.cmap("+compileF(f)+",(Bag)"+compileM(s)+")";
- case AggregateMap(`f,`acc,`zero,`s):
- return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)
- +","+compileM(#<cMap(`f,`s)>)+"))";
- case MapReduce(`m,`r,`s,_):
- return "MapReduceAlgebra.mapReduce("+compileF(m)+","
- +compileF(r)+",(Bag)"+compileM(s)+")";
- case MapAggregateReduce(`m,`r,`acc,`zero,`s,_):
- return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)
- +","+compileE(zero)+","+compileM(#<MapReduce(`m,`r,`s)>)+"))";
- case MapCombineReduce(`m,`c,`r,`s,_):
- return "MapReduceAlgebra.mapReduce("+compileF(m)
- +","+compileF(r)+",(Bag)"+compileM(s)+")";
- case MapReduce2(`mx,`my,`c,`r,`x,`y,_):
- return "MapReduceAlgebra.mapReduce2("+compileF(mx)+","+compileF(my)
- +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
- case MapReduce2(`mx,`my,`r,`x,`y,_):
- return "MapReduceAlgebra.mapReduce2("+compileF(mx)+","+compileF(my)
- +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
- case MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,_):
- return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)
- +","+compileM(#< MapReduce2(`mx,`my,`r,`x,`y)>)+"))";
- case MapJoin(`kx,`ky,`r,`x,`y):
- return "MapReduceAlgebra.mapJoin("+compileF(kx)+","+compileF(ky)
- +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
- case MapAggregateJoin(`kx,`ky,`r,`acc,`zero,`x,`y):
- return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)
- +","+compileE(zero)+","+compileM(#<MapJoin(`kx,`ky,`r,`x,`y)>)+"))";
- case GroupByJoin(`kx,`ky,`gx,`gy,`m,`c,`r,`x,`y,`o):
- return "MapReduceAlgebra.groupByJoin("+compileF(kx)+","+compileF(ky)
- +","+compileF(gx)+","+compileF(gy)+","+compileF(m)+","+compileF(c)
- +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
- case CrossProduct(`mx,`my,`r,`x,`y):
- return "MapReduceAlgebra.crossProduct("+compileF(mx)+","+compileF(my)
- +","+compileF(r)+",(Bag)"+compileM(x)+",(Bag)"+compileM(y)+")";
- case CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y):
- return "new Bag(MapReduceAlgebra.aggregate("+compileF(acc)+","+compileE(zero)
- +","+compileM(#<CrossProduct(`mx,`my,`r,`x,`y)>)+"))";
- case BSPSource(`n,BinarySource(`file,_)):
- if (Config.hadoop_mode)
- return "Plan.binarySource("+compileE(n)+",((MR_string)"
- +compileE(file)+").get())";
- else return "(Bag)MapReduceAlgebra.read_binary("+compileE(n)+",((MR_string)"
- +compileE(file)+").get())";
- case BinarySource(`file,_):
- if (Config.hadoop_mode)
- return "Plan.binarySource(((MR_string)"+compileE(file)+").get())";
- else return "(Bag)MapReduceAlgebra.read_binary(((MR_string)"+compileE(file)+").get())";
- case BSPSource(`n,ParsedSource(`parser,`file,...args)):
- if (!(n instanceof LongLeaf))
- fail;
- if (!Config.hadoop_mode)
- return "MapReduceAlgebra.parsedSource(((MR_int)"+compileE(n)+").get(),\""
- +parser+"\",((MR_string)"+compileE(file)+").get(),"
- +reify(args)+")";
- Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- return "Plan.parsedSource("+p.getName()+".class,((MR_string)"+compileE(file)+").get(),"
- +reify(args)+")";
- case ParsedSource(`parser,`file,...args):
- if (!Config.hadoop_mode)
- return "MapReduceAlgebra.parsedSource(\""+parser+"\",((MR_string)"
- +compileE(file)+").get(),"+reify(args)+")";
- Class<? extends Parser> p = DataSource.parserDirectory.get(parser.toString());
- if (p == null)
- throw new Error("Unknown parser: "+parser);
- return "Plan.parsedSource("+p.getName()+".class,((MR_string)"+compileE(file)+").get(),"
- +reify(args)+")";
- case Merge(`x,`y):
- return "((Bag)"+compileM(x)+").union((Bag)"+compileM(y)+")";
- case Generator(`min,`max,`size):
- return "MapReduceAlgebra.generator(((MR_long)"+compileE(min)+").get(),"
- +"((MR_long)"+compileE(max)+").get())";
- case BSP(`n,`superstep,`state,`o,...as):
- String ds = "";
- for ( Tree a: as )
- ds += ",(Bag)("+compileM(a)+")";
- return "MapReduceAlgebra.BSP("+((LongLeaf)n).value()+","
- +compileF(superstep)+","+compileE(state)+","+o+","
- +"new Bag[]{"+ds.substring(1)+"})";
- case `v:
- if (v.is_variable())
- return v.toString();
- };
- throw new Exception("Cannot compile: "+e);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Config.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Config.java b/src/main/java/core/Config.java
deleted file mode 100644
index a957d89..0000000
--- a/src/main/java/core/Config.java
+++ /dev/null
@@ -1,277 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.VariableLeaf;
-import java.util.ArrayList;
-import java.io.FileInputStream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-
-/** MRQL configuration parameters */
-final public class Config {
- public static boolean loaded = false;
-
- // true for using Hadoop HDFS file-system
- public static boolean hadoop_mode = false;
- // true for local execution (one node)
- public static boolean local_mode = false;
- // true for local execution (one node)
- public static boolean distributed_mode = false;
- // true for Hadoop map-reduce mode
- public static boolean map_reduce_mode = false;
- // true, for BSP mode using Hama
- public static boolean bsp_mode = false;
- // true, for Spark mode
- public static boolean spark_mode = false;
- // if true, it process the input interactively
- public static boolean interactive = true;
- // compile the MR functional arguments to Java bytecode at run-time
- // (each task-tracker repeats the compilation at the MR setup time)
- public static boolean compile_functional_arguments = true;
- // if true, generates info about all compilation and optimization steps
- public static boolean trace = false;
- // number of worker nodes
- public static int nodes = 2;
- // true, to disable mapJoin
- public static boolean noMapJoin = false;
- // max distributed cache size for MapJoin (fragment-replicate join) in MBs
- public static int mapjoin_size = 50;
- // max entries for in-mapper combiner before they are flushed out
- public static int map_cache_size = 100000;
- // max number of bag elements to print
- public static int max_bag_size_print = 20;
- // max size of materialized vector before is spilled to a file:
- public static int max_materialized_bag = 500000;
- // max number of incoming messages before a sub-sync()
- public static int bsp_msg_size = Integer.MAX_VALUE;
- // number of elements per mapper to process the range min...max
- public static long range_split_size = 100000;
- // max number of streams to merge simultaneously
- public static int max_merged_streams = 100;
- // the directory for temporary files and spilled bags
- public static String tmpDirectory = "/tmp/mrql_"+System.getProperty("user.name");
- // true, if we want to derive a combine function for MapReduce
- public static boolean use_combiner = true;
- // true, if we can use the rule that fuses a groupBy with a join over the same key
- public static boolean groupJoinOpt = true;
- // true, if we can use the rule that converts a self-join into a simple mapreduce
- public static boolean selfJoinOpt = true;
- // true for run-time trace of plans
- public static boolean trace_execution = false;
- // true for extensive run-time trace of expressions & plans
- public static boolean trace_exp_execution = false;
- // true if you don't want to print statistics
- public static boolean quiet_execution = false;
- // true if this is during testing
- public static boolean testing = false;
- // true to display INFO log messages
- public static boolean info = false;
-
- /** store the configuration parameters */
- public static void write ( Configuration conf ) {
- conf.setBoolean("mrql.hadoop.mode",hadoop_mode);
- conf.setBoolean("mrql.local.mode",local_mode);
- conf.setBoolean("mrql.distributed.mode",distributed_mode);
- conf.setBoolean("mrql.map.reduce.mode",map_reduce_mode);
- conf.setBoolean("mrql.bsp.mode",bsp_mode);
- conf.setBoolean("mrql.spark.mode",spark_mode);
- conf.setBoolean("mrql.interactive",interactive);
- conf.setBoolean("mrql.compile.functional.arguments",compile_functional_arguments);
- conf.setBoolean("mrql.trace",trace);
- conf.setInt("mrql.nodes",nodes);
- conf.setInt("mrql.mapjoin.size",mapjoin_size);
- conf.setInt("mrql.in.mapper.size",map_cache_size);
- conf.setInt("mrql.max.bag.size.print",max_bag_size_print);
- conf.setInt("mrql.max.materialized.bag",max_materialized_bag);
- conf.setInt("mrql.bsp.msg.size",bsp_msg_size);
- conf.setLong("mrql.range.split.size",range_split_size);
- conf.setInt("mrql.max.merged.streams",max_merged_streams);
- conf.set("mrql.tmp.directory",tmpDirectory);
- conf.setBoolean("mrql.use.combiner",use_combiner);
- conf.setBoolean("mrql.group.join.opt",groupJoinOpt);
- conf.setBoolean("mrql.self.join.opt",selfJoinOpt);
- conf.setBoolean("mrql.trace.execution",trace_execution);
- conf.setBoolean("mrql.trace.exp.execution",trace_exp_execution);
- conf.setBoolean("mrql.quiet.execution",quiet_execution);
- conf.setBoolean("mrql.testing",testing);
- conf.setBoolean("mrql.info",info);
- }
-
- /** load the configuration parameters */
- public static void read ( Configuration conf ) {
- if (loaded)
- return;
- loaded = true;
- hadoop_mode = conf.getBoolean("mrql.hadoop.mode",hadoop_mode);
- local_mode = conf.getBoolean("mrql.local.mode",local_mode);
- distributed_mode = conf.getBoolean("mrql.distributed.mode",distributed_mode);
- map_reduce_mode = conf.getBoolean("mrql.map.reduce.mode",map_reduce_mode);
- bsp_mode = conf.getBoolean("mrql.bsp.mode",bsp_mode);
- spark_mode = conf.getBoolean("mrql.spark.mode",spark_mode);
- interactive = conf.getBoolean("mrql.interactive",interactive);
- compile_functional_arguments = conf.getBoolean("mrql.compile.functional.arguments",compile_functional_arguments);
- trace = conf.getBoolean("mrql.trace",trace);
- nodes = conf.getInt("mrql.nodes",nodes);
- mapjoin_size = conf.getInt("mrql.mapjoin.size",mapjoin_size);
- map_cache_size = conf.getInt("mrql.in.mapper.size",map_cache_size);
- max_bag_size_print = conf.getInt("mrql.max.bag.size.print",max_bag_size_print);
- max_materialized_bag = conf.getInt("mrql.max.materialized.bag",max_materialized_bag);
- bsp_msg_size = conf.getInt("mrql.bsp.msg.size",bsp_msg_size);
- range_split_size = conf.getLong("mrql.range.split.size",range_split_size);
- max_merged_streams = conf.getInt("mrql.max.merged.streams",max_merged_streams);
- tmpDirectory = conf.get("mrql.tmp.directory");
- use_combiner = conf.getBoolean("mrql.use.combiner",use_combiner);
- groupJoinOpt = conf.getBoolean("mrql.group.join.opt",groupJoinOpt);
- selfJoinOpt = conf.getBoolean("mrql.self.join.opt",selfJoinOpt);
- trace_execution = conf.getBoolean("mrql.trace.execution",trace_execution);
- trace_exp_execution = conf.getBoolean("mrql.trace.exp.execution",trace_exp_execution);
- quiet_execution = conf.getBoolean("mrql.quiet.execution",quiet_execution);
- testing = conf.getBoolean("mrql.testing",testing);
- info = conf.getBoolean("mrql.info",info);
- }
-
- public static ArrayList<String> extra_args = new ArrayList<String>();
-
- /** read configuration parameters from the Main args */
- public static Bag parse_args ( String args[], Configuration conf ) throws Exception {
- int i = 0;
- int iargs = 0;
- extra_args = new ArrayList<String>();
- ClassImporter.load_classes();
- interactive = true;
- while (i < args.length) {
- if (args[i].equals("-local")) {
- local_mode = true;
- i++;
- } else if (args[i].equals("-dist")) {
- distributed_mode = true;
- i++;
- } else if (args[i].equals("-reducers")) {
- if (++i >= args.length)
- throw new Error("Expected number of reductions");
- nodes = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-bsp")) {
- bsp_mode = true;
- i++;
- } else if (args[i].equals("-spark")) {
- spark_mode = true;
- i++;
- } else if (args[i].equals("-bsp_tasks")) {
- if (++i >= args.length && Integer.parseInt(args[i]) < 1)
- throw new Error("Expected max number of bsp tasks > 1");
- nodes = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-nodes")) {
- if (++i >= args.length && Integer.parseInt(args[i]) < 1)
- throw new Error("Expected number of nodes > 1");
- nodes = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-bsp_msg_size")) {
- if (++i >= args.length && Integer.parseInt(args[i]) < 10000)
- throw new Error("Expected max number of bsp messages before subsync() > 10000");
- bsp_msg_size = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-mapjoin_size")) {
- if (++i >= args.length)
- throw new Error("Expected number of MBs");
- mapjoin_size = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-cache_size")) {
- if (++i >= args.length)
- throw new Error("Expected number of entries");
- map_cache_size = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-tmp")) {
- if (++i >= args.length)
- throw new Error("Expected a temporary directory");
- tmpDirectory = args[i];
- i++;
- } else if (args[i].equals("-bag_size")) {
- if (++i >= args.length && Integer.parseInt(args[i]) < 10000)
- throw new Error("Expected max size of materialized bag > 10000");
- max_materialized_bag = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-bag_print")) {
- if (++i >= args.length)
- throw new Error("Expected number of bag elements to print");
- max_bag_size_print = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-split_size")) {
- if (++i >= args.length)
- throw new Error("Expected a split size");
- range_split_size = Long.parseLong(args[i]);
- i++;
- } else if (args[i].equals("-max_merged")) {
- if (++i >= args.length)
- throw new Error("Expected a max number of merged streams");
- max_merged_streams = Integer.parseInt(args[i]);
- i++;
- } else if (args[i].equals("-trace")) {
- trace = true;
- i++;
- } else if (args[i].equals("-C")) {
- compile_functional_arguments = true;
- i++;
- } else if (args[i].equals("-NC")) {
- compile_functional_arguments = false;
- i++;
- } else if (args[i].equals("-P")) {
- trace_execution = true;
- i++;
- } else if (args[i].equals("-quiet")) {
- quiet_execution = true;
- i++;
- } else if (args[i].equals("-info")) {
- info = true;
- i++;
- } else if (args[i].equals("-trace_execution")) {
- trace_execution = true;
- trace_exp_execution = true;
- compile_functional_arguments = false;
- i++;
- } else if (args[i].equals("-methods")) {
- System.out.print("\nImported methods: ");
- ClassImporter.print_methods();
- System.out.println();
- System.out.print("\nAggregations:");
- Translator.print_aggregates();
- System.out.println();
- i++;
- } else if (args[i].charAt(0) == '-')
- throw new Error("Unknown MRQL parameter: "+args[i]);
- else {
- if (interactive) {
- Main.query_file = args[i++];
- interactive = false;
- } else extra_args.add(args[i++]);
- }
- };
- if (hadoop_mode)
- write(conf);
- Plan.conf = conf;
- Bag b = new Bag();
- for ( String s: extra_args )
- b.add(new MR_string(s));
- Interpreter.new_global_binding(new VariableLeaf("args").value(),b);
- return b;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/DataSet.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/DataSet.java b/src/main/java/core/DataSet.java
deleted file mode 100644
index efe5646..0000000
--- a/src/main/java/core/DataSet.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.util.List;
-import java.util.ArrayList;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** The domain of the MRQL physical algebra is a set of DataSources */
-public class DataSet {
- public ArrayList<DataSource> source; // multiple sources
- public long counter; // a Hadoop user-defined counter used by the `repeat' operator
- public long records; // total number of dataset records
-
- /** Construct a DataSet that contains one DataSource
- * @param s the given DataSource
- * @param counter a Hadoop user-defined counter used by the `repeat' operator
- * @param records total number of dataset records
- */
- DataSet ( DataSource s, long counter, long records ) {
- source = new ArrayList<DataSource>();
- source.add(s);
- this.counter = counter;
- this.records = records;
- }
-
- /** Construct a set of DataSources
- * @param counter a Hadoop user-defined counter used by the `repeat' operator
- * @param records total number of dataset records
- */
- DataSet ( long counter, long records ) {
- source = new ArrayList<DataSource>();
- this.counter = counter;
- this.records = records;
- }
-
- /** add a DataSource to this DataSet */
- public void add ( DataSource s ) {
- source.add(s);
- }
-
- /** merge this DataSet with the given DataSet */
- public void merge ( DataSet ds ) {
- source.addAll(ds.source);
- counter += ds.counter;
- records += ds.records;
- }
-
- /** dataset size in bytes */
- public long size ( Configuration conf ) {
- long n = 0;
- for (DataSource s: source)
- n += s.size(conf);
- return n;
- }
-
- /** return a single DataSource path by merging all the DataSource paths in this DataSet */
- public String merge () {
- Object[] ds = source.toArray();
- String path = ((DataSource)ds[0]).path.toString();
- for ( int i = 1; i < ds.length; i++ )
- path += ","+((DataSource)ds[i]).path;
- return path;
- }
-
- /** return the first num values */
- public List<MRData> take ( int num ) {
- int count = num;
- ArrayList<MRData> res = new ArrayList<MRData>();
- for ( DataSource s: source ) {
- res.addAll(s.take(count));
- if (res.size() < count)
- count = count-res.size();
- else return res;
- };
- return res;
- }
-
- /** accumulate all dataset values */
- public MRData reduce ( MRData zero, Function acc ) {
- MRData res = zero;
- for ( DataSource s: source )
- res = s.reduce(res,acc);
- return res;
- }
-
- public String toString () {
- String p = "<"+counter;
- for (DataSource s: source)
- p += ","+s;
- return p+">";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/DataSource.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/DataSource.java b/src/main/java/core/DataSource.java
deleted file mode 100644
index 6a0ef4f..0000000
--- a/src/main/java/core/DataSource.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.Iterator;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.conf.Configuration;
-
-
-/** A DataSource is any input data source, such as a text file, a key/value map, a data base, an intermediate file, etc */
-public class DataSource {
- final public static String separator = "%%%";
- public static DataSourceDirectory dataSourceDirectory = new DataSourceDirectory();
- public static ParserDirectory parserDirectory = new ParserDirectory();
- private static boolean loaded = false;
-
- public String path;
- public Class<? extends MRQLFileInputFormat> inputFormat;
- public int source_num;
- public boolean to_be_merged; // if the path is a directory with multiple files, merge them
-
- final static class ParserDirectory extends HashMap<String,Class<? extends Parser>> {
- }
-
- /** A dictionary that maps data source paths to DataSource data.
- * It assumes that each path can only be associated with a single data source format and parser
- */
- final static class DataSourceDirectory extends HashMap<String,DataSource> {
- public void read ( Configuration conf ) {
- clear();
- for ( String s: conf.get("mrql.data.source.directory").split("@@@") ) {
- String[] p = s.split("===");
- put(p[0],DataSource.read(p[1],conf));
- }
- }
-
- public String toString () {
- String s = "";
- for ( String k: keySet() )
- s += "@@@"+k+"==="+get(k);
- if (s.equals(""))
- return s;
- else return s.substring(3);
- }
-
- public DataSource get ( String name ) {
- for ( Map.Entry<String,DataSource> e: entrySet() )
- if (name.startsWith(e.getKey()))
- return e.getValue();
- return null;
- }
-
- public void distribute ( Configuration conf ) {
- conf.set("mrql.data.source.directory",toString());
- }
- }
-
- DataSource () {}
-
- DataSource ( int source_num,
- String path,
- Class<? extends MRQLFileInputFormat> inputFormat,
- Configuration conf ) {
- this.source_num = source_num;
- this.path = path;
- this.inputFormat = inputFormat;
- to_be_merged = false;
- try {
- Path p = new Path(path);
- FileSystem fs = p.getFileSystem(conf);
- String complete_path = fs.getFileStatus(p).getPath().toString();
- //String complete_path = "file:"+path;
- this.path = complete_path;
- dataSourceDirectory.put(this.path,this);
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- public static void loadParsers() {
- if (!loaded) {
- DataSource.parserDirectory.put("xml",XMLParser.class);
- DataSource.parserDirectory.put("json",JsonFormatParser.class);
- DataSource.parserDirectory.put("line",LineParser.class);
- loaded = true;
- }
- }
-
- static {
- loadParsers();
- }
-
- private static long size ( Path path, Configuration conf ) throws IOException {
- FileStatus s = path.getFileSystem(conf).getFileStatus(path);
- if (!s.isDir())
- return s.getLen();
- long size = 0;
- for ( FileStatus fs: path.getFileSystem(conf).listStatus(path) )
- size += fs.getLen();
- return size;
- }
-
- /** data set size in bytes */
- public long size ( Configuration conf ) {
- try {
- return size(new Path(path),conf);
- } catch (IOException e) {
- throw new Error(e);
- }
- }
-
- public static DataSource read ( String buffer, Configuration conf ) {
- try {
- String[] s = buffer.split(separator);
- int n = Integer.parseInt(s[1]);
- if (s[0].equals("Binary"))
- return new BinaryDataSource(n,s[2],conf);
- else if (s[0].equals("Generator"))
- return new GeneratorDataSource(n,s[2],conf);
- else if (s[0].equals("Text"))
- return new ParsedDataSource(n,s[3],parserDirectory.get(s[2]),((Node)Tree.parse(s[4])).children(),conf);
- else throw new Error("Unrecognized data source: "+s[0]);
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- public static DataSource get ( String path, Configuration conf ) {
- if (dataSourceDirectory.isEmpty())
- dataSourceDirectory.read(conf);
- return dataSourceDirectory.get(path);
- }
-
- public static DataSource getCached ( String remote_path, String local_path, Configuration conf ) {
- DataSource ds = get(remote_path,conf);
- ds.path = local_path;
- dataSourceDirectory.put(local_path,ds);
- return ds;
- }
-
- /** return the first num values */
- public List<MRData> take ( int num ) {
- int count = num;
- try {
- ArrayList<MRData> res = new ArrayList<MRData>();
- Iterator<MRData> it = inputFormat.newInstance().materialize(new Path(path)).iterator();
- for ( int i = num; (num < 0 || i > 0) && it.hasNext(); i-- )
- if (Config.hadoop_mode && Config.bsp_mode)
- res.add(((Tuple)it.next()).get(1)); // strip tag in BSP mode
- else res.add(it.next());
- return res;
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
-
- static Tuple tuple_container = new Tuple(new Tuple(),new Tuple());
-
- /** accumulate all datasource values */
- public MRData reduce ( MRData zero, final Function acc ) {
- try {
- MRData res = zero;
- for ( MRData x: inputFormat.newInstance().materialize(new Path(path)) ) {
- if (Config.hadoop_mode && Config.bsp_mode)
- x = ((Tuple)x).get(1); // strip tag in BSP mode
- tuple_container.set(0,res).set(1,x);
- res = acc.eval(tuple_container);
- };
- return res;
- } catch (Exception ex) {
- throw new Error(ex);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/Environment.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/Environment.java b/src/main/java/core/Environment.java
deleted file mode 100644
index da8dd84..0000000
--- a/src/main/java/core/Environment.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java.io.*;
-import org.apache.mrql.gen.Tree;
-
-
-/** the run-time environment for in-memory evaluation (binds variables to MRData) */
-final public class Environment implements Serializable {
- public String name;
- public MRData value;
- public Environment next;
-
- Environment ( String n, MRData v, Environment next ) {
- name = n;
- value = v;
- this.next = next;
- }
-
- private void writeObject(ObjectOutputStream out) throws IOException {
- out.defaultWriteObject();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- name = Tree.add(name);
- }
-}
[22/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MRData.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MRData.java b/core/src/main/java/org/apache/mrql/MRData.java
new file mode 100644
index 0000000..8c08269
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MRData.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.hadoop.io.WritableComparable;
+import java.io.*;
+
+
+/** All MRQL data are encoded as MRData (similar to AVRO form) */
+public abstract class MRData implements WritableComparable<MRData>, Serializable {
+ abstract public void materializeAll ();
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MRQL.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MRQL.java b/core/src/main/java/org/apache/mrql/MRQL.java
new file mode 100644
index 0000000..7ba081e
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MRQL.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+
+
+/** API for dynamic MRQL queries */
+final public class MRQL extends Interpreter {
+
+ /** evaluate an MRQL query in a string
+ * @param query a string that contains an MRQL query
+ * @return the evaluation result
+ */
+ public static MRData query ( String query ) {
+ evaluate("store tt := "+query+";");
+ return variable_lookup("tt",global_env);
+ }
+
+ /** evaluate MRQL statments in a string
+ * @param command a string that contains MRQL commands separated by ;
+ */
+ public static void evaluate ( String command ) {
+ try {
+ MRQLParser parser = new MRQLParser();
+ parser.setScanner(new MRQLLex(new StringReader(command)));
+ MRQLLex.reset();
+ parser.parse();
+ } catch (Exception x) {
+ x.printStackTrace();
+ throw new Error(x);
+ }
+ }
+
+ /** clean up the MRQL workspace */
+ public static void clean () {
+ try {
+ Plan.clean();
+ } catch (IOException ex) {
+ throw new Error("Failed to clean-up temporary files");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MRQLFileInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MRQLFileInputFormat.java b/core/src/main/java/org/apache/mrql/MRQLFileInputFormat.java
new file mode 100644
index 0000000..6b6aec2
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MRQLFileInputFormat.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+
+
+/** An interface for all MRQL FileInputFormats */
+public interface MRQLFileInputFormat {
+ /** materialize the input file into a memory Bag */
+ public Bag materialize ( final Path path ) throws IOException;
+
+ /** materialize the entire dataset into a Bag
+ * @param x the DataSet in HDFS to collect values from
+ * @param strip is not used in MapReduce mode
+ * @return the Bag that contains the collected values
+ */
+ public Bag collect ( final DataSet x, boolean strip ) throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_bool.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_bool.java b/core/src/main/java/org/apache/mrql/MR_bool.java
new file mode 100644
index 0000000..d40573f
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_bool.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+/** a container for boolean values */
+final public class MR_bool extends MRData {
+ private boolean value;
+
+ public MR_bool ( boolean i ) { value = i; }
+
+ public void materializeAll () {};
+
+ public boolean get () { return value; }
+
+ public void set ( boolean v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.BOOLEAN);
+ out.writeBoolean(value);
+ }
+
+ final public static MR_bool read ( DataInput in ) throws IOException {
+ return new MR_bool(in.readBoolean());
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = in.readBoolean();
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_bool);
+ return (value == ((MR_bool) x).value) ? 0 : (value ? 1 : -1);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ size[0] = 2;
+ boolean bx = x[xs] > 0;
+ boolean by = y[ys] > 0;
+ return (bx == by) ? 0 : (bx ? 1 : -1);
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_bool && ((MR_bool)x).value==value;
+ }
+
+ public int hashCode () { return value ? 0 : 1; }
+
+ public String toString () {
+ return Boolean.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_byte.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_byte.java b/core/src/main/java/org/apache/mrql/MR_byte.java
new file mode 100644
index 0000000..e5c319e
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_byte.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for byte values */
+final public class MR_byte extends MRData {
+ private byte value;
+
+ public MR_byte ( byte i ) { value = i; }
+ public MR_byte ( int i ) { value = (byte)i; }
+
+ public void materializeAll () {};
+
+ public byte get () { return value; }
+
+ public void set ( byte v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.BYTE);
+ out.writeByte(value);
+ }
+
+ final public static MR_byte read ( DataInput in ) throws IOException {
+ return new MR_byte(in.readByte());
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = in.readByte();
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_byte);
+ byte v = ((MR_byte) x).value;
+ return (value == v) ? 0 : ((value < v) ? -1 : 1);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ size[0] = 2;
+ return x[xs]-y[ys];
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_byte && ((MR_byte)x).value==value;
+ }
+
+ public int hashCode () { return Math.abs(value); }
+
+ public String toString () {
+ return Integer.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_char.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_char.java b/core/src/main/java/org/apache/mrql/MR_char.java
new file mode 100644
index 0000000..f0e9721
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_char.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for a char */
+final public class MR_char extends MRData {
+ private char value;
+
+ public MR_char ( char i ) { value = i; }
+
+ public void materializeAll () {};
+
+ public char get () { return value; }
+
+ public void set ( char v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.CHAR);
+ out.writeChar(value);
+ }
+
+ final public static MR_char read ( DataInput in ) throws IOException {
+ return new MR_char(in.readChar());
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = in.readChar();
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_char);
+ return value-((MR_char) x).value;
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ size[0] = 2;
+ return x[xs]-y[ys];
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_char && ((MR_char)x).value==value;
+ }
+
+ public int hashCode () { return value; }
+
+ public String toString () {
+ return Character.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_dataset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_dataset.java b/core/src/main/java/org/apache/mrql/MR_dataset.java
new file mode 100644
index 0000000..abc81cd
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_dataset.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a wrapper of a DataSet (stored in HDFS) as an MRData */
+final public class MR_dataset extends MRData {
+ public DataSet dataset;
+
+ public MR_dataset ( DataSet d ) { dataset = d; }
+
+ public void materializeAll () {};
+
+ public DataSet dataset () { return dataset; }
+
+ public void write ( DataOutput out ) throws IOException {
+ throw new Error("DataSets are not serializable");
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ throw new Error("DataSets are not serializable");
+ }
+
+ public int compareTo ( MRData x ) {
+ throw new Error("DataSets cannot be compared");
+ }
+
+ public boolean equals ( Object x ) {
+ throw new Error("DataSets cannot be compared");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_double.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_double.java b/core/src/main/java/org/apache/mrql/MR_double.java
new file mode 100644
index 0000000..567917c
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_double.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for double values */
+final public class MR_double extends MRData {
+ private double value;
+
+ public MR_double ( double x ) { value = x; }
+
+ public void materializeAll () {};
+
+ public double get () { return value; }
+
+ public void set ( double v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.DOUBLE);
+ out.writeDouble(value);
+ }
+
+ final public static MR_double read ( DataInput in ) throws IOException {
+ return new MR_double(in.readDouble());
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = in.readDouble();
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_double);
+ double v = ((MR_double) x).value;
+ return (value == v) ? 0 : ((value > v) ? 1 : -1);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ size[0] = (Double.SIZE >> 3)+1;
+ double v = WritableComparator.readDouble(x,xs) - WritableComparator.readDouble(y,ys);
+ return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_double && ((MR_double)x).value==value;
+ }
+
+ public int hashCode () {
+ return Math.abs((int)Double.doubleToLongBits(value));
+ }
+
+ public String toString () {
+ return Double.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_float.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_float.java b/core/src/main/java/org/apache/mrql/MR_float.java
new file mode 100644
index 0000000..d8dda38
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_float.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for float values */
+final public class MR_float extends MRData {
+ private float value;
+
+ public MR_float ( float x ) { value = x; }
+
+ public MR_float ( double x ) { value = (float)x; }
+
+ public void materializeAll () {};
+
+ public float get () { return value; }
+
+ public void set ( float v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.FLOAT);
+ out.writeFloat(value);
+ }
+
+ final public static MR_float read ( DataInput in ) throws IOException {
+ return new MR_float(in.readFloat());
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = in.readFloat();
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_float);
+ float v = ((MR_float) x).value;
+ return (value == v) ? 0 : ((value > v) ? 1 : -1);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ size[0] = (Float.SIZE >> 3)+1;
+ float v = WritableComparator.readFloat(x,xs) - WritableComparator.readFloat(y,ys);
+ return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_float && ((MR_float)x).value==value;
+ }
+
+ public int hashCode () {
+ return Math.abs(Float.floatToIntBits(value));
+ }
+
+ public String toString () {
+ return Float.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_int.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_int.java b/core/src/main/java/org/apache/mrql/MR_int.java
new file mode 100644
index 0000000..6586fe0
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_int.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for int values */
+final public class MR_int extends MRData {
+ private int value;
+
+ public MR_int ( int i ) { value = i; }
+
+ public void materializeAll () {};
+
+ public int get () { return value; }
+
+ public void set ( int v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.INT);
+ WritableUtils.writeVInt(out,value);
+ }
+
+ final public static MR_int read ( DataInput in ) throws IOException {
+ return new MR_int(WritableUtils.readVInt(in));
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = WritableUtils.readVInt(in);
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_int);
+ return value - ((MR_int) x).value;
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ try {
+ size[0] = 1+WritableUtils.decodeVIntSize(x[xs]);
+ int v = WritableComparator.readVInt(x,xs)-WritableComparator.readVInt(y,ys);
+ return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_int && ((MR_int)x).value==value;
+ }
+
+ public int hashCode () { return Math.abs(value); }
+
+ public String toString () {
+ return Integer.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_long.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_long.java b/core/src/main/java/org/apache/mrql/MR_long.java
new file mode 100644
index 0000000..2078048
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_long.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for long values */
+final public class MR_long extends MRData {
+ private long value;
+
+ public MR_long ( long i ) { value = i; }
+
+ public void materializeAll () {};
+
+ public long get () { return value; }
+
+ public void set ( long v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.LONG);
+ WritableUtils.writeVLong(out,value);
+ }
+
+ final public static MR_long read ( DataInput in ) throws IOException {
+ return new MR_long(WritableUtils.readVLong(in));
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = WritableUtils.readVLong(in);
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_long);
+ long v = ((MR_long) x).value;
+ return (value == v) ? 0 : ((value > v) ? 1 : -1);
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ try {
+ size[0] = 1+WritableUtils.decodeVIntSize(x[xs]);
+ long v = WritableComparator.readVLong(x,xs)-WritableComparator.readVLong(y,ys);
+ return (v == 0) ? 0 : ((v > 0) ? 1 : -1);
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_long && ((MR_long)x).value==value;
+ }
+
+ public int hashCode () { return (int)Math.abs(value); }
+
+ public String toString () {
+ return Long.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_more_bsp_steps.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_more_bsp_steps.java b/core/src/main/java/org/apache/mrql/MR_more_bsp_steps.java
new file mode 100644
index 0000000..0d844ad
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_more_bsp_steps.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+
+/** used for BSP synchronization when a peer needs to do more steps */
+final public class MR_more_bsp_steps extends MRData {
+ MR_more_bsp_steps () {}
+
+ public void materializeAll () {};
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.MORE_BSP_STEPS);
+ }
+
+ public void readFields ( DataInput in ) throws IOException {}
+
+ public int compareTo ( MRData x ) { return 0; }
+
+ public boolean equals ( Object x ) { return x instanceof MR_more_bsp_steps; }
+
+ public int hashCode () { return 0; }
+
+ public String toString () { return "more_bsp_steps"; }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_short.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_short.java b/core/src/main/java/org/apache/mrql/MR_short.java
new file mode 100644
index 0000000..c50e124
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_short.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for short values */
+final public class MR_short extends MRData {
+ private short value;
+
+ public MR_short ( short i ) { value = i; }
+
+ public void materializeAll () {};
+
+ public short get () { return value; }
+
+ public void set ( short v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.SHORT);
+ out.writeShort(value);
+ }
+
+ final public static MR_short read ( DataInput in ) throws IOException {
+ return new MR_short(in.readShort());
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = in.readShort();
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_short);
+ return value - ((MR_short) x).value;
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ size[0] = (Short.SIZE >> 3)+1;
+ return WritableComparator.readUnsignedShort(x,xs) - WritableComparator.readUnsignedShort(y,ys);
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_short && ((MR_short)x).value==value;
+ }
+
+ public int hashCode () { return Math.abs((int)value); }
+
+ public String toString () {
+ return Short.toString(value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_string.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_string.java b/core/src/main/java/org/apache/mrql/MR_string.java
new file mode 100644
index 0000000..a914cca
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_string.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for strings */
+final public class MR_string extends MRData {
+ private String value;
+
+ public MR_string ( String s ) { value = s; }
+
+ public void materializeAll () {};
+
+ public String get () { return value; }
+
+ public void set ( String v ) { value = v; }
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.STRING);
+ Text.writeString(out,value);
+ }
+
+ final public static MR_string read ( DataInput in ) throws IOException {
+ return new MR_string(Text.readString(in));
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ value = Text.readString(in);
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof MR_string);
+ return value.compareTo(((MR_string) x).value);
+ }
+
+ final static Text.Comparator comparator = new Text.Comparator();
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ try {
+ size[0] = 1+WritableComparator.readVInt(x,xs)+WritableUtils.decodeVIntSize(x[xs]);
+ return comparator.compare(x,xs,xl,y,ys,yl);
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ public boolean equals ( Object x ) {
+ return x instanceof MR_string && value.equals(((MR_string) x).value);
+ }
+
+ public int hashCode () { return value.hashCode(); }
+
+ public String toString () {
+ return "\""+value+"\"";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_sync.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_sync.java b/core/src/main/java/org/apache/mrql/MR_sync.java
new file mode 100644
index 0000000..275414f
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_sync.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+
+/** used for BSP synchronization */
+final public class MR_sync extends MRData {
+ MR_sync () {}
+
+ public void materializeAll () {};
+
+ final public void write ( DataOutput out ) throws IOException {
+ out.writeByte(MRContainer.SYNC);
+ }
+
+ public void readFields ( DataInput in ) throws IOException {}
+
+ public int compareTo ( MRData x ) { return 0; }
+
+ public boolean equals ( Object x ) { return x instanceof MR_sync; }
+
+ public int hashCode () { return 0; }
+
+ public String toString () { return "sync"; }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MR_variable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MR_variable.java b/core/src/main/java/org/apache/mrql/MR_variable.java
new file mode 100644
index 0000000..f81f117
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MR_variable.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+
+/** a template variable; should appear only in a template */
+final public class MR_variable extends MRData {
+ public int var_num;
+
+ MR_variable ( int n ) { var_num = n; }
+
+ public void materializeAll () {};
+
+ final public void write ( DataOutput out ) throws IOException {}
+
+ public void readFields ( DataInput in ) throws IOException {}
+
+ public int compareTo ( MRData x ) { return 0; }
+
+ public boolean equals ( Object x ) { return false; }
+
+ public int hashCode () { return 0; }
+
+ public String toString () {
+ return "variable("+var_num+")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Main.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Main.java b/core/src/main/java/org/apache/mrql/Main.java
new file mode 100644
index 0000000..6e0f527
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Main.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import java.util.Enumeration;
+import org.apache.hadoop.util.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.*;
+import jline.*;
+
+
+final public class Main {
+ public final static String version = "0.9.0";
+
+ public static PrintStream print_stream;
+ public static Configuration conf;
+ static MRQLParser parser = new MRQLParser();
+ public static String query_file = "";
+
+ public static void include_file ( String file ) {
+ try {
+ MRQLParser old_parser = parser;
+ boolean old_interactive = Config.interactive;
+ try {
+ parser = new MRQLParser(new MRQLLex(new FileInputStream(file)));
+ } catch (Exception e) {
+ Path path = new Path(file);
+ FileSystem fs = path.getFileSystem(conf);
+ parser = new MRQLParser(new MRQLLex(fs.open(path)));
+ };
+ Config.interactive = false;
+ parser.parse();
+ Config.interactive = old_interactive;
+ parser = old_parser;
+ } catch (Exception ex) {
+ ex.printStackTrace(System.err);
+ throw new Error(ex);
+ }
+ }
+
+ public static void main ( String[] args ) throws Exception {
+ boolean hadoop = false;
+ for ( String arg: args ) {
+ hadoop |= arg.equals("-local") || arg.equals("-dist");
+ Config.bsp_mode |= arg.equals("-bsp");
+ Config.spark_mode |= arg.equals("-spark");
+ };
+ Config.map_reduce_mode = !Config.bsp_mode && !Config.spark_mode;
+ if (Config.map_reduce_mode)
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.MapReduceEvaluator").newInstance();
+ if (Config.bsp_mode)
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
+ if (Config.spark_mode)
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.SparkEvaluator").newInstance();
+ if (hadoop) {
+ conf = Evaluator.evaluator.new_configuration();
+ GenericOptionsParser gop = new GenericOptionsParser(conf,args);
+ conf = gop.getConfiguration();
+ args = gop.getRemainingArgs();
+ };
+ Config.parse_args(args,conf);
+ Config.hadoop_mode = Config.local_mode || Config.distributed_mode;
+ if (!Config.info) {
+ for ( Enumeration en = LogManager.getCurrentLoggers(); en.hasMoreElements(); )
+ ((Logger)en.nextElement()).setLevel(Level.WARN);
+ LogManager.getRootLogger().setLevel(Level.WARN);
+ };
+ Evaluator.evaluator.init(conf);
+ new TopLevel();
+ System.out.print("Apache MRQL version "+version+" (");
+ if (Config.compile_functional_arguments)
+ System.out.print("compiled ");
+ else System.out.print("interpreted ");
+ if (hadoop) {
+ if (Config.local_mode)
+ System.out.print("local ");
+ else if (Config.distributed_mode)
+ System.out.print("distributed ");
+ if (Config.spark_mode)
+ System.out.println("Spark mode using "+Config.nodes+" tasks)");
+ else if (Config.bsp_mode)
+ System.out.println("Hama BSP mode over "+Config.nodes+" BSP tasks)");
+ else if (Config.nodes > 0)
+ System.out.println("Hadoop MapReduce mode with "+Config.nodes+" reducers)");
+ else if (!Config.local_mode)
+ System.out.println("Hadoop MapReduce mode with 1 reducer, use -nodes to change it)");
+ else System.out.println("Hadoop MapReduce mode)");
+ } else if (Config.bsp_mode)
+ System.out.println("in-memory BSP mode)");
+ else System.out.println("in-memory MapReduce mode)");
+ if (Config.interactive) {
+ System.out.println("Type quit to exit");
+ ConsoleReader reader = new ConsoleReader();
+ reader.setBellEnabled(false);
+ History history = new History(new File(System.getProperty("user.home")+"/.mrqlhistory"));
+ reader.setHistory(history);
+ reader.setUseHistory(false);
+ try {
+ loop: while (true) {
+ String line = "";
+ String s = "";
+ try {
+ if (hadoop && Config.bsp_mode)
+ Config.write(Plan.conf);
+ do {
+ s = reader.readLine("> ");
+ if (s != null && (s.equals("quit") || s.equals("exit")))
+ break loop;
+ if (s != null)
+ line += " "+s;
+ } while (s == null || s.indexOf(";") <= 0);
+ line = line.substring(1);
+ history.addToHistory(line);
+ parser = new MRQLParser(new MRQLLex(new StringReader(line)));
+ MRQLLex.reset();
+ parser.parse();
+ } catch (EOFException x) {
+ break;
+ } catch (Exception x) {
+ if (x.getMessage() != null)
+ System.out.println(x);
+ } catch (Error x) {
+ System.out.println(x);
+ }
+ }
+ } finally {
+ if (hadoop) {
+ Plan.clean();
+ Evaluator.evaluator.shutdown(Plan.conf);
+ };
+ if (Config.compile_functional_arguments)
+ Compiler.clean();
+ }
+ } else try {
+ if (hadoop && Config.bsp_mode)
+ Config.write(Plan.conf);
+ try {
+ parser = new MRQLParser(new MRQLLex(new FileInputStream(query_file)));
+ } catch (Exception e) {
+ // when the query file is in HDFS
+ Path path = new Path(query_file);
+ FileSystem fs = path.getFileSystem(conf);
+ parser = new MRQLParser(new MRQLLex(fs.open(path)));
+ };
+ parser.parse();
+ } finally {
+ if (hadoop) {
+ Plan.clean();
+ Evaluator.evaluator.shutdown(Plan.conf);
+ };
+ if (Config.compile_functional_arguments)
+ Compiler.clean();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MapReduceAlgebra.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MapReduceAlgebra.java b/core/src/main/java/org/apache/mrql/MapReduceAlgebra.java
new file mode 100644
index 0000000..198a533
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MapReduceAlgebra.java
@@ -0,0 +1,810 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.*;
+
+
+/** Evaluation of MRQL algebra expressions in memory */
+final public class MapReduceAlgebra {
+
+ /** eager concat-map (not used) */
+ private static Bag cmap_eager ( final Function f, final Bag s ) {
+ Bag res = new Bag();
+ for ( MRData e: s )
+ res.addAll((Bag)f.eval(e));
+ return res;
+ }
+
+ /** lazy concat-map (stream-based)
+ * @param f a function from a to {b}
+ * @param s the input of type {a}
+ * @return a value of type {b}
+ */
+ public static Bag cmap ( final Function f, final Bag s ) {
+ final Iterator<MRData> si = s.iterator();
+ return new Bag(new BagIterator() {
+ Iterator<MRData> data = null;
+ boolean more = false;
+ public boolean hasNext () {
+ if (data == null) {
+ while (!more && si.hasNext()) {
+ data = ((Bag)f.eval(si.next())).iterator();
+ more = data.hasNext();
+ }
+ } else {
+ if (more) {
+ more = data.hasNext();
+ if (more)
+ return true;
+ };
+ while (!more && si.hasNext()) {
+ data = ((Bag)f.eval(si.next())).iterator();
+ more = data.hasNext();
+ }
+ };
+ return more;
+ }
+ public MRData next () {
+ return data.next();
+ }
+ });
+ }
+
+ /** lazy map
+ * @param f a function from a to b
+ * @param s the input of type {a}
+ * @return a value of type {b}
+ */
+ public static Bag map ( final Function f, final Bag s ) {
+ final Iterator<MRData> si = s.iterator();
+ return new Bag(new BagIterator() {
+ public boolean hasNext () { return si.hasNext(); }
+ public MRData next () { return f.eval(si.next()); }
+ });
+ }
+
+ /** lazy filter combined with a map
+ * @param p a function from a to boolean
+ * @param f a function from a to b
+ * @param s the input of type {a}
+ * @return a value of type {b}
+ */
+ public static Bag filter ( final Function p, final Function f, final Bag s ) {
+ final Iterator<MRData> si = s.iterator();
+ return new Bag(new BagIterator() {
+ MRData data = null;
+ public boolean hasNext () {
+ while (si.hasNext()) {
+ data = si.next();
+ if (((MR_bool)p.eval(data)).get())
+ return true;
+ };
+ return false;
+ }
+ public MRData next () { return f.eval(data); }
+ });
+ }
+
+ /** strict group-by
+ * @param s the input of type {(a,b)}
+ * @return a value of type {(a,{b})}
+ */
+ public static Bag groupBy ( Bag s ) {
+ Bag res = new Bag();
+ s.sort();
+ MRData last = null;
+ Bag group = new Bag();
+ for ( MRData e: s) {
+ final Tuple p = (Tuple)e;
+ if (last != null && p.first().equals(last))
+ group.add(p.second());
+ else {
+ if (last != null) {
+ group.trim();
+ res.add(new Tuple(last,group));
+ };
+ last = p.first();
+ group = new Bag();
+ group.add(p.second());
+ }
+ };
+ if (last != null) {
+ group.trim();
+ res.add(new Tuple(last,group));
+ };
+ //res.trim();
+ return res;
+ }
+
+ /** lazy group-by (not used) */
+ private static Bag groupBy_lazy ( Bag s ) {
+ s.sort();
+ final Iterator<MRData> it = s.iterator();
+ return new Bag(new BagIterator() {
+ MRData last = null;
+ MRData data = null;
+ Bag group = new Bag();
+ public boolean hasNext () {
+ while (it.hasNext()) {
+ final Tuple p = (Tuple)it.next();
+ if (last != null && p.first().equals(last))
+ group.add(p.second());
+ else if (last != null) {
+ group.trim();
+ data = new Tuple(last,group);
+ last = p.first();
+ group = new Bag();
+ group.add(p.second());
+ return true;
+ } else {
+ last = p.first();
+ group = new Bag();
+ group.add(p.second());
+ }
+ };
+ if (last != null) {
+ group.trim();
+ data = new Tuple(last,group);
+ last = null;
+ return true;
+ };
+ return false;
+ }
+ public MRData next () {
+ return data;
+ }
+ });
+ }
+
+ /** the MapReduce operation
+ * @param m a map function from a to {(k,b)}
+ * @param r a reduce function from (k,{b}) to {c}
+ * @param s the input of type {a}
+ * @return a value of type {c}
+ */
+ public static Bag mapReduce ( final Function m, final Function r, final Bag s ) {
+ return cmap(r,groupBy(cmap(m,s)));
+ }
+
+ /** Not used: use mapReduce2 instead */
+ private static Bag join ( final Function kx, final Function ky, final Function f,
+ final Bag X, final Bag Y ) {
+ return cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple p = (Tuple)e;
+ return (Bag)f.eval(new Tuple(p.second(),
+ cmap(new Function() {
+ public Bag eval ( final MRData y ) {
+ return (ky.eval(y).equals(p.first()))
+ ? new Bag(y)
+ : new Bag();
+ } }, Y))); }
+ },
+ groupBy(cmap(new Function() {
+ public Bag eval ( final MRData x ) {
+ return new Bag(new Tuple(kx.eval(x),x));
+ } }, X)));
+ }
+
+ /** A hash-based equi-join
+ * @param kx left key function from a to k
+ * @param ky right key function from b to k
+ * @param f reducer from (a,b) to c
+ * @param X left input of type {a}
+ * @param Y right input of type {b}
+ * @return a value of type {c}
+ */
+ public static Bag hash_join ( final Function kx, final Function ky, final Function f,
+ final Bag X, final Bag Y ) {
+ Hashtable<MRData,Bag> hashTable = new Hashtable<MRData,Bag>(1000);
+ for ( MRData x: X ) {
+ MRData key = kx.eval(x);
+ Bag old = hashTable.get(key);
+ if (old == null)
+ hashTable.put(key,new Bag(x));
+ else old.add(x);
+ };
+ Bag res = new Bag();
+ for ( MRData y: Y ) {
+ MRData key = ky.eval(y);
+ Bag match = hashTable.get(key);
+ if (match != null)
+ for ( MRData x: match )
+ res.add(f.eval(new Tuple(x,y)));
+ };
+ return res;
+ }
+
+ /** A cross-product
+ * @param mx left map function from a to {a'}
+ * @param my right key function from b to {b'}
+ * @param r reducer from (a',b') to {c}
+ * @param X left input of type {a}
+ * @param Y right input of type {b}
+ * @return a value of type {c}
+ */
+ public static Bag crossProduct ( final Function mx, final Function my, final Function r,
+ final Bag X, final Bag Y ) {
+ Bag a = new Bag();
+ for ( MRData y: Y )
+ for ( MRData v: (Bag)my.eval(y) )
+ a.add(v);
+ Bag b = new Bag();
+ for ( MRData x: X )
+ for ( MRData xx: (Bag)mx.eval(x) )
+ for ( MRData y: a )
+ for ( MRData v: (Bag)r.eval(new Tuple(xx,y)) )
+ b.add(v);
+ return b;
+ }
+
+ /** A map-reduce operation with two mappers (a join)
+ * @param mx left map function from a to {(k,a')}
+ * @param my right key function from b to {(k,b')}
+ * @param r reducer from ({a'},{b'}) to {c}
+ * @param X left input of type {a}
+ * @param Y right input of type {b}
+ * @return a value of type {c}
+ */
+ public static Bag mapReduce2 ( final Function mx, // left mapper
+ final Function my, // right mapper
+ final Function r, // reducer
+ final Bag X, final Bag Y ) {
+ final Bag left = cmap(new Function() {
+ public Bag eval ( final MRData x ) {
+ return cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple p = (Tuple)e;
+ return new Bag(new Tuple(p.first(),
+ new Tuple(new MR_byte(1),p.second())));
+ } }, (Bag)mx.eval(x));
+ } }, X);
+ final Bag right = cmap(new Function() {
+ public Bag eval ( final MRData y ) {
+ return cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple p = (Tuple)e;
+ return new Bag(new Tuple(p.first(),
+ new Tuple(new MR_byte(2),p.second())));
+ } }, (Bag)my.eval(y));
+ } }, Y);
+ final Iterator<MRData> li = left.iterator();
+ final Iterator<MRData> ri = right.iterator();
+ final Bag mix = new Bag(new BagIterator () {
+ MRData data;
+ public boolean hasNext () {
+ if (li.hasNext()) {
+ data = li.next();
+ return true;
+ } else if (ri.hasNext()) {
+ data = ri.next();
+ return true;
+ } else return false;
+ }
+ public MRData next () {
+ return data;
+ }
+ });
+ return cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple p = (Tuple)e;
+ final Bag xs = cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple q = (Tuple)e;
+ return (((MR_byte)q.first()).get() == 1)
+ ? new Bag(q.second())
+ : new Bag();
+ } }, (Bag)p.second());
+ final Bag ys = cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple q = (Tuple)e;
+ return (((MR_byte)q.first()).get() == 2)
+ ? new Bag(q.second())
+ : new Bag();
+ } }, (Bag)p.second());
+ xs.materialize();
+ ys.materialize();
+ return (Bag)r.eval(new Tuple(xs,ys));
+ } }, groupBy(mix));
+ }
+
+ /** The fragment-replicate join (map-side join)
+ * @param kx left key function from a to k
+ * @param ky right key function from b to k
+ * @param r reducer from (a,{b}) to {c}
+ * @param X left input of type {a}
+ * @param Y right input of type {b}
+ * @return a value of type {c}
+ */
+ public static Bag mapJoin ( final Function kx, final Function ky, final Function r,
+ final Bag X, final Bag Y ) {
+ X.materialize();
+ Y.materialize();
+ return cmap(new Function() {
+ public Bag eval ( final MRData e ) {
+ final Tuple p = (Tuple)e;
+ return cmap(new Function() {
+ public Bag eval ( final MRData x ) {
+ return (kx.eval(x).equals(p.first()))
+ ? (Bag)r.eval(new Tuple(x,p.second()))
+ : new Bag();
+ } }, X); }
+ },
+ groupBy(cmap(new Function() {
+ public Bag eval ( final MRData y ) {
+ return new Bag(new Tuple(ky.eval(y),y));
+ } }, Y)));
+ }
+
+ /** An equi-join combined with a group-by (see GroupByJoinPlan)
+ * @param kx left key function from a to k
+ * @param ky right key function from b to k
+ * @param gx group-by key function from a to k1
+ * @param gy group-by key function from b to k2
+ * @param m mapper from (a,b) to {c}
+ * @param c combiner from ((k1,k2),{c}) to d
+ * @param r reducer from ((k1,k2),d) to {e}
+ * @param X left input of type {a}
+ * @param Y right input of type {b}
+ * @return a value of type {e}
+ */
+ public static Bag groupByJoin ( final Function kx, final Function ky,
+ final Function gx, final Function gy,
+ final Function m, final Function c, final Function r,
+ final Bag X, final Bag Y ) {
+ Bag s = groupBy(hash_join(kx,ky,
+ new Function() {
+ public MRData eval ( final MRData e ) {
+ Tuple t = (Tuple)e;
+ return new Tuple(new Tuple(gx.eval(t.first()),gy.eval(t.second())),t);
+ } },
+ X,Y));
+ Bag res = new Bag();
+ for ( MRData z: s ) {
+ Tuple t = (Tuple)z;
+ for ( MRData n: (Bag)r.eval(new Tuple(t.first(),c.eval(new Tuple(t.first(),cmap(m,(Bag)t.second()))))) )
+ res.add(n);
+ };
+ return res;
+ }
+
+ private static void flush_table ( final Map<MRData,MRData> hashTable, final Function r, final Bag result ) {
+ Bag tbag = new Bag(2);
+ Tuple pair = new Tuple(2);
+ for ( Map.Entry<MRData,MRData> entry: hashTable.entrySet() ) {
+ pair.set(0,entry.getKey());
+ tbag.clear();
+ tbag.add_element(entry.getValue());
+ pair.set(1,tbag);
+ for ( MRData e: (Bag)r.eval(pair) )
+ result.add(e);
+ };
+ hashTable.clear();
+ }
+
+ /** An equi-join combined with a group-by implemented using hashing
+ * @param kx left key function from a to k
+ * @param ky right key function from b to k
+ * @param gx group-by key function from a to k1
+ * @param gy group-by key function from b to k2
+ * @param m mapper from (a,b) to {c}
+ * @param c combiner from ((k1,k2),{c}) to d
+ * @param r reducer from ((k1,k2),d) to {e}
+ * @param X left input of type {a}
+ * @param Y right input of type {b}
+ * @return a value of type {e}
+ */
+ final public static Bag mergeGroupByJoin ( final Function kx, final Function ky,
+ final Function gx, final Function gy,
+ final Function m, final Function c, final Function r,
+ Bag X, Bag Y ) {
+ Bag tbag = new Bag(2);
+ Tuple pair = new Tuple(2);
+ Tuple vpair = new Tuple(2);
+ final Map<MRData,MRData> hashTable = new HashMap<MRData,MRData>(1000);
+ Bag xs = groupBy(map(new Function() {
+ public MRData eval ( final MRData e ) {
+ Tuple t = (Tuple)e;
+ return new Tuple(new Tuple(t.first(),kx.eval(t.second())),t.second());
+ } }, X));
+ Bag ys = groupBy(map(new Function() {
+ public MRData eval ( final MRData e ) {
+ Tuple t = (Tuple)e;
+ return new Tuple(new Tuple(t.first(),ky.eval(t.second())),t.second());
+ } }, Y));
+ X = null; Y = null;
+ Bag res = new Bag();
+ final Iterator<MRData> xi = xs.iterator();
+ final Iterator<MRData> yi = ys.iterator();
+ if ( !xi.hasNext() || !yi.hasNext() )
+ return res;
+ Tuple x = (Tuple)xi.next();
+ Tuple y = (Tuple)yi.next();
+ MRData partition = null;
+ while ( xi.hasNext() && yi.hasNext() ) {
+ int cmp = x.first().compareTo(y.first());
+ if (cmp < 0) { x = (Tuple)xi.next(); continue; };
+ if (cmp > 0) { y = (Tuple)yi.next(); continue; };
+ if (partition == null)
+ partition = ((Tuple)x.first()).first();
+ else if (!partition.equals(((Tuple)x.first()).first())) {
+ partition = ((Tuple)x.first()).first();
+ flush_table(hashTable,r,res);
+ };
+ for ( MRData xx: (Bag)x.second() )
+ for ( MRData yy: (Bag)y.second() ) {
+ Tuple key = new Tuple(gx.eval(xx),gy.eval(yy));
+ vpair.set(0,xx).set(1,yy);
+ MRData old = hashTable.get(key);
+ pair.set(0,key);
+ for ( MRData e: (Bag)m.eval(vpair) )
+ if (old == null)
+ hashTable.put(key,e);
+ else {
+ tbag.clear();
+ tbag.add_element(e).add_element(old);
+ pair.set(1,tbag);
+ for ( MRData z: (Bag)c.eval(pair) )
+ hashTable.put(key,z); // normally, done once
+ }
+ };
+ if (xi.hasNext())
+ x = (Tuple)xi.next();
+ if (yi.hasNext())
+ y = (Tuple)yi.next();
+ };
+ flush_table(hashTable,r,res);
+ return res;
+ }
+
+ /** repeat the loop until all termination conditions are true or until we reach the max num of steps
+ * @param loop a function from {a} to {(a,boolean)}
+ * @param init the initial value of type {a}
+ * @param max_num the maximum number of steps
+ * @return a value of type {a}
+ */
+ public static Bag repeat ( final Function loop,
+ final Bag init,
+ final int max_num ) throws Exception {
+ boolean cont;
+ int i = 0;
+ Bag s = init;
+ s.materializeAll();
+ do {
+ MRData d = loop.eval(s);
+ i++;
+ cont = false;
+ if (d instanceof Bag) {
+ Bag bag = (Bag) d;
+ bag.materialize();
+ s.clear();
+ for ( MRData x: bag ) {
+ Tuple t = (Tuple)x;
+ cont |= ((MR_bool)t.second()).get();
+ s.add(t.first());
+ }
+ } else if (d instanceof MR_dataset) {
+ DataSet ds = ((MR_dataset)d).dataset();
+ if (ds.counter != 0)
+ cont = true;
+ System.err.println("*** Repeat #"+i+": "+ds.counter+" true results");
+ s = Plan.collect(ds);
+ } else throw new Error("Wrong repeat");
+ } while (cont && i <= max_num);
+ return s;
+ }
+
+ /** transitive closure: repeat the loop until the new set is equal to the previous set
+ * or until we reach the max num of steps
+ * @param loop a function from {a} to {a}
+ * @param init the initial value of type {a}
+ * @param max_num the maximum number of steps
+ * @return a value of type {a}
+ */
+ public static Bag closure ( final Function loop,
+ final Bag init,
+ final int max_num ) throws Exception {
+ int i = 0;
+ long n = 0;
+ long old = 0;
+ Bag s = init;
+ s.materializeAll();
+ do {
+ MRData d = loop.eval(s);
+ i++;
+ if (d instanceof Bag) {
+ s = (Bag)d;
+ s.materialize();
+ old = n;
+ n = s.size();
+ } else if (d instanceof MR_dataset) {
+ DataSet ds = ((MR_dataset)d).dataset();
+ System.err.println("*** Repeat #"+i+": "+(ds.records-n)+" new records");
+ old = n;
+ n = ds.records;
+ s = Plan.collect(ds);
+ } else throw new Error("Wrong repeat");
+ } while (old < n && i <= max_num);
+ return s;
+ }
+
+ /** parse a text document using a given parser
+ * @param parser the parser
+ * @param file the text document (local file)
+ * @param args the arguments to pass to the parser
+ * @return a lazy bag that contains the parsed data
+ */
+ public static Bag parsedSource ( final Parser parser,
+ final String file,
+ Trees args ) {
+ try {
+ parser.initialize(args);
+ parser.open(file);
+ return new Bag(new BagIterator() {
+ Iterator<MRData> result = null;
+ MRData data;
+ public boolean hasNext () {
+ try {
+ while (result == null || !result.hasNext()) {
+ String s = parser.slice();
+ if (s == null)
+ return false;
+ result = parser.parse(s).iterator();
+ };
+ data = (MRData)result.next();
+ return true;
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+ public MRData next () {
+ return data;
+ }
+ });
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ /** parse a text document using a given parser
+ * @param parser the name of the parser
+ * @param file the text document (local file)
+ * @param args the arguments to pass to the parser
+ * @return a lazy bag that contains the parsed data
+ */
+ public static Bag parsedSource ( String parser, String file, Trees args ) {
+ try {
+ return parsedSource(DataSource.parserDirectory.get(parser).newInstance(),file,args);
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ private static Bag add_source_num ( int source_num, Bag input ) {
+ return new Bag(new Tuple(new MR_int(source_num),input));
+ }
+
+ /** parse a text document using a given parser and tag output data with a source num
+ * @param source_num the source id
+ * @param parser the parser
+ * @param file the text document (local file)
+ * @param args the arguments to pass to the parser
+ * @return a lazy bag that contains the parsed data taged with the source id
+ */
+ public static Bag parsedSource ( int source_num,
+ Parser parser,
+ String file,
+ Trees args ) {
+ return add_source_num(source_num,parsedSource(parser,file,args));
+ }
+
+ /** parse a text document using a given parser and tag output data with a source num
+ * @param source_num the source id
+ * @param parser the name of the parser
+ * @param file the text document (local file)
+ * @param args the arguments to pass to the parser
+ * @return a lazy bag that contains the parsed data taged with the source id
+ */
+ public static Bag parsedSource ( int source_num, String parser, String file, Trees args ) {
+ try {
+ return parsedSource(source_num,DataSource.parserDirectory.get(parser).newInstance(),file,args);
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ /** aggregate the Bag elements
+ * @param accumulator a function from (b,a) to b
+ * @param zero a value of type b
+ * @param s a Bag of type {a}
+ * @return a value of type b
+ */
+ public static MRData aggregate ( final Function accumulator,
+ final MRData zero,
+ final Bag s ) {
+ MRData result = zero;
+ for ( MRData x: s )
+ result = accumulator.eval(new Tuple(result,x));
+ return result;
+ }
+
+ public static MRData materialize ( MRData x ) {
+ if (x instanceof Bag)
+ ((Bag)x).materialize();
+ return x;
+ }
+
+ /** Dump the value of some type to a binary local file;
+ * The type is dumped to a separate file.type
+ */
+ public static void dump ( String file, Tree type, MRData value ) throws IOException {
+ PrintStream ftp = new PrintStream(file+".type");
+ ftp.print("1@"+type.toString()+"\n");
+ ftp.close();
+ DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(file)));
+ value.write(out);
+ out.close();
+ }
+
+ /** return the type of the dumped binary local file from file.type */
+ public static Tree get_type ( String file ) {
+ try {
+ BufferedReader ftp = new BufferedReader(new FileReader(new File(file+".type")));
+ String s[] = ftp.readLine().split("@");
+ ftp.close();
+ if (s.length != 2)
+ return null;
+ if (!s[0].equals("1"))
+ throw new Error("The binary file has been created in hadoop mode and cannot be read in java mode");
+ return Tree.parse(s[1]);
+ } catch (Exception e) {
+ return null;
+ }
+ }
+
+ /** read the contents of a dumped local binary file */
+ public static MRData read_binary ( String file ) {
+ try {
+ Tree type = get_type(file);
+ DataInputStream in = new DataInputStream(new FileInputStream(new File(file)));
+ return MRContainer.read(in);
+ } catch (Exception e) {
+ return null;
+ }
+ }
+
+ /** read the contents of a dumped local binary file and tag data with a source num */
+ public static Bag read_binary ( int source_num, String file ) {
+ return add_source_num(source_num,(Bag)read_binary(file));
+ }
+
+ /** generate a lazy bag of long numbers {min...max} */
+ public static Bag generator ( final long min, final long max ) {
+ if (min > max)
+ throw new Error("Min value ("+min+") is larger than max ("+max+") in generator");
+ return new Bag(new BagIterator() {
+ long index = min;
+ public boolean hasNext () {
+ return index <= max;
+ }
+ public MRData next () {
+ return new MR_long(index++);
+ }
+ });
+ }
+
+ /** generate a lazy bag of long numbers {min...max} and tag each lon number with a source num */
+ public static Bag generator ( int source_num, final long min, final long max ) {
+ return add_source_num(source_num,generator(min,max));
+ }
+
+ /** the cache that holds all local data in memory */
+ private static Tuple cache;
+
+ /** return the cache element at location loc */
+ public static MRData getCache ( int loc ) {
+ return cache.get(loc);
+ }
+
+ /** set the cache element at location loc to value and return ret */
+ public static MRData setCache ( int loc, MRData value, MRData ret ) {
+ if (value instanceof Bag)
+ materialize((Bag)value);
+ cache.set(loc,value);
+ return ret;
+ }
+
+ /** The BSP operation
+ * @param source the source ids of the input Bags
+ * @param superstep the BSP superstep is a function from ({M},S) to ({M},S,boolean)
+ * @param init_state is the initial state of type S
+ * @param order do we need to order the result?
+ * @param inputs the input Bags
+ * @return return a Bag in cache[0]
+ */
+ public static MRData BSP ( final int[] source,
+ final Function superstep,
+ final MRData init_state,
+ boolean order,
+ final Bag[] inputs ) {
+ Bag msgs = new Bag();
+ MRData state = init_state;
+ Tuple result;
+ boolean exit;
+ boolean skip = false;
+ String tabs = "";
+ int step = 0;
+ cache = new Tuple(100);
+ for ( int i = 0; i < 100; i++ )
+ cache.set(i,new Bag());
+ for ( Bag x: inputs ) {
+ Tuple p = (Tuple)(x.get(0));
+ cache.set(((MR_int)p.first()).get(),
+ materialize(p.second()));
+ };
+ do {
+ if (!skip)
+ step++;
+ if (!skip && Config.trace_execution) {
+ tabs = Interpreter.tabs(Interpreter.tab_count);
+ System.out.println(tabs+" Superstep "+step+":");
+ System.out.println(tabs+" messages: "+msgs);
+ System.out.println(tabs+" state: "+state);
+ for ( int i = 0; i < cache.size(); i++)
+ if (cache.get(i) instanceof Bag && ((Bag)cache.get(i)).size() > 0)
+ System.out.println(tabs+" cache "+i+": "+cache.get(i));
+ };
+ result = (Tuple)superstep.eval(new Tuple(cache,msgs,state,new MR_string("")));
+ Bag new_msgs = (Bag)result.get(0);
+ state = result.get(1);
+ exit = ((MR_bool)result.get(2)).get();
+ skip = new_msgs == SystemFunctions.bsp_empty_bag;
+ if ((!skip || exit) && Config.trace_execution)
+ System.out.println(tabs+" result: "+result);
+ final Iterator<MRData> iter = new_msgs.iterator();
+ msgs = new Bag(new BagIterator() {
+ public boolean hasNext () {
+ return iter.hasNext();
+ }
+ public MRData next () {
+ return ((Tuple)iter.next()).get(1);
+ }
+ });
+ } while (!exit);
+ MRData[] data = new MRData[source.length];
+ for ( int i = 0; i < data.length; i++ )
+ data[i] = getCache(source[i]);
+ if (order && data[0] instanceof Bag) {
+ final Iterator<MRData> iter = ((Bag)data[0]).iterator();
+ return new Bag(new BagIterator() {
+ public boolean hasNext () {
+ return iter.hasNext();
+ }
+ public MRData next () {
+ return ((Tuple)iter.next()).get(0);
+ }
+ });
+ };
+ if (data.length == 1)
+ return data[0];
+ else return new Tuple(data);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Materialization.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Materialization.gen b/core/src/main/java/org/apache/mrql/Materialization.gen
new file mode 100644
index 0000000..5683699
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Materialization.gen
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.*;
+
+
+class Domains {
+ public Trees domains;
+ public Trees repeats;
+ public Domains ( Trees d, Trees r ) { domains = d; repeats = r; }
+}
+
+
+/** if the plan refers to a variable bound to a stream-based Bag and occurs in the code
+ * multiple times, embed code to materialize this Bag in memory
+ */
+final public class Materialization extends Translator {
+
+ // is this a direct-access term? (not the results of a bulk operation)
+ private static boolean access_variable ( Tree e ) {
+ match e {
+ case nth(`x,_):
+ return access_variable(x);
+ case union_value(`x):
+ return access_variable(x);
+ case index(`x,`n):
+ return access_variable(x);
+ case `v:
+ if (v.is_variable())
+ return true;
+ };
+ return false;
+ }
+
+ private static Domains new_domain ( Trees vars, Tree e, Domains d ) {
+ if (!access_variable(e))
+ return materialize(vars,e,d);
+ Domains nd = new Domains(d.domains,d.repeats);
+ if ((d.domains.member(e) || !free_variables(e,vars).is_empty())
+ && !d.repeats.member(e))
+ nd.repeats = nd.repeats.cons(e);
+ nd.domains = nd.domains.cons(e);
+ return nd;
+ }
+
+ private static Domains union ( Domains xd, Domains yd ) {
+ Domains nd = new Domains(xd.domains,xd.repeats);
+ for ( Tree y: yd.domains )
+ if (!nd.domains.member(y))
+ nd.domains = nd.domains.cons(y);
+ for ( Tree y: yd.repeats )
+ if (!nd.repeats.member(y))
+ nd.repeats = nd.repeats.cons(y);
+ return nd;
+ }
+
+ final static int unionM = ClassImporter.find_method_number("plus",#[bag(any),bag(any)]);
+
+ private static Domains materialize ( Trees vars, Tree e, Domains d ) {
+ match e {
+ case lambda(`v,`b):
+ return materialize(#[`v],b,d);
+ case cmap(lambda(`v,`b),`s):
+ return materialize(#[`v],b,new_domain(vars,s,d));
+ case map(lambda(`v,`b),`s):
+ return materialize(#[`v],b,new_domain(vars,s,d));
+ case filter(lambda(`v1,`b1),lambda(`v2,`b2),`s):
+ return materialize(#[`v1],b1,materialize(#[`v2],b2,new_domain(vars,s,d)));
+ case aggregate(lambda(`v,`b),`z,`s):
+ return materialize(#[`v],b,new_domain(vars,s,d));
+ case groupBy(`s):
+ return new_domain(vars,s,d);
+ case orderBy(`s):
+ return new_domain(vars,s,d);
+ case mapReduce2(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`v,`b),`x,`y,`o):
+ return materialize(#[`v],b,materialize(#[`v1],b1,materialize(#[`v2],b2,
+ new_domain(vars,x,new_domain(vars,y,d)))));
+ case join(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`v,`b),`x,`y):
+ return materialize(#[`v],b,materialize(#[`v1],b1,materialize(#[`v2],b2,
+ new_domain(vars,x,new_domain(vars,y,d)))));
+ case crossProduct(lambda(`v1,`b1),lambda(`v2,`b2),lambda(`v,`b),`x,`y):
+ return materialize(#[`v],b,materialize(#[`v1],b1,materialize(#[`v2],b2,
+ new_domain(vars,x,new_domain(vars,y,d)))));
+ case let(`v,`x,`y):
+ Domains nd = materialize(vars.cons(v),y,materialize(vars,x,d));
+ Trees zs = #[];
+ for ( Tree z: nd.repeats )
+ if (!v.equals(z))
+ zs = zs.cons(z);
+ nd.repeats = zs;
+ return nd;
+ case if(`p,`x,`y):
+ Domains nd = materialize(vars,p,d);
+ return union(materialize(vars,x,nd),
+ materialize(vars,y,nd));
+ case callM(union,_,`x,`y):
+ return new_domain(vars,x,new_domain(vars,y,d));
+ case callM(_,`k,`x,`y):
+ if (((LongLeaf)k).value() != unionM)
+ fail;
+ return new_domain(vars,x,new_domain(vars,y,d));
+ case `f(...as):
+ Domains nd = new Domains(d.domains,d.repeats);
+ for ( Tree a: as )
+ nd = materialize(vars,a,nd);
+ return nd;
+ };
+ return d;
+ }
+
+ public static Tree materialize_terms ( Tree e ) {
+ Domains d = materialize(#[],e,new Domains(#[],#[]));
+ for ( Tree x: d.repeats )
+ e = subst(x,#<materialize(`x)>,e);
+ return e;
+ }
+}
[21/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/MethodInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/MethodInfo.java b/core/src/main/java/org/apache/mrql/MethodInfo.java
new file mode 100644
index 0000000..2dc9ad1
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/MethodInfo.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.lang.reflect.Method;
+import org.apache.mrql.gen.*;
+
+
+/** class for storing Java method information */
+final public class MethodInfo implements Comparable<MethodInfo> {
+ public String name;
+ public Trees signature;
+ public Method method;
+
+ MethodInfo ( String n, Trees s, Method m ) {
+ name = n;
+ signature = s;
+ method = m;
+ }
+
+ public int compareTo ( MethodInfo x ) {
+ int c = name.compareTo(x.name);
+ if (c != 0)
+ return c;
+ if (signature.length() < x.signature.length())
+ return -1;
+ if (signature.length() > x.signature.length())
+ return 1;
+ // handles overloading: more specific method signatures first
+ for ( int i = 1; i < signature.length(); i++ ) {
+ int ct = TypeInference.compare_types(signature.nth(i),x.signature.nth(i));
+ if (ct != 0)
+ return ct;
+ };
+ return TypeInference.compare_types(signature.nth(0),x.signature.nth(0));
+ }
+
+ public boolean equals ( Object x ) {
+ return name.equals(((MethodInfo)x).name)
+ && signature.equals(((MethodInfo)x).signature);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Normalization.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Normalization.gen b/core/src/main/java/org/apache/mrql/Normalization.gen
new file mode 100644
index 0000000..37d628e
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Normalization.gen
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+
+
+/** normalize algebraic expressions to more efficient forms using heuristic rules */
+public class Normalization extends Translator {
+
+ /** given that pattern=e, find the bindings of the pattern variables */
+ static Trees bind_pattern ( Tree pattern, Tree e ) {
+ Trees args = #[];
+ match pattern {
+ case tuple(...pl):
+ int i = 0;
+ for ( Tree p: pl ) {
+ args = args.append(bind_pattern(p,#<nth(`e,`i)>));
+ i++;
+ }
+ case record(...bl):
+ Trees attrs = #[];
+ for ( Tree b: bl )
+ match b {
+ case bind(`n,`p):
+ args = args.append(bind_pattern(p,#<project(`e,`n)>));
+ if (attrs.member(n))
+ error("Duplicate record attribute name: "+n);
+ attrs = attrs.append(n);
+ };
+ case typed(`p,`t):
+ args = bind_pattern(p,#<typed(`e,`t)>);
+ case list(...pl):
+ int i = 0;
+ for ( Tree p: pl ) {
+ args = args.append(bind_pattern(p,#<index(`e,`i)>));
+ i++;
+ };
+ args = args.append(#<call(eq,call(count,`e),`i)>);
+ case call(`c,...s):
+ Tree ci = data_constructors.lookup(c.toString());
+ if (ci == null)
+ error("Undefined data constructor: "+c);
+ match ci {
+ case `dname(`n,`tp):
+ args = args.append(#<call(eq,union_tag(`e),`n)>);
+ args = args.append(bind_pattern(s.length() == 1 ? s.head() : #<tuple(...s)>,
+ #<typed(union_value(`e),`tp)>));
+ };
+ case any: ;
+ case `v:
+ if (!v.is_variable()) // constant in pattern
+ args = #[call(eq,`e,`v)];
+ else if (st.lookup(v.toString()) != null // repeated pattern variable
+ && !(e.is_variable() && st.lookup(v.toString()).is_variable())) // exception
+ args = #[call(eq,`e,`(st.lookup(v.toString())))];
+ else st.insert(v.toString(),e); // new pattern variable
+ };
+ return args;
+ }
+
+ private static Tree make_tuple ( Trees pl ) {
+ if (pl.length() == 1)
+ return pl.head();
+ return #<tuple(...pl)>;
+ }
+
+ /** remove group-bys and order-bys from the MRQL queries */
+ static Tree remove_groupby ( Tree e ) {
+ Tree ret = #<error>;
+ match e {
+ case select(distinct,`u,from(...bl),where(`c),groupby(...gl),orderby(...ol)):
+ ret = #<select(none,tuple(`u,`u),from(...bl),where(`c),groupby(...gl),orderby(...ol))>;
+ ret = #<cmap(lambda(tuple(key,group),list(key)),groupBy(`ret))>;
+ return remove_groupby(ret);
+ case select(none,`u,from(...bl),where(`c),groupby(),orderby()):
+ return remove_groupby(#<select(`u,from(...bl),where(`c))>);
+ case select(none,`u,from(...bl),where(`c),groupby(...gl),orderby(`l,...ol)):
+ Tree tol = make_tuple(ol);
+ ret = #<cmap(lambda(tuple(key,group),group),
+ orderBy(select(none,tuple(`tol,`u),
+ from(...bl),
+ where(`c),groupby(...gl),orderby())))>;
+ return (l.equals(#<none>))
+ ? remove_groupby(ret)
+ : #<range(`(remove_groupby(ret)),0,`l)>;
+ case select(none,`u,from(...bl),where(`c),groupby(`h,...gl),orderby()):
+ Trees pl = #[];
+ Trees ul = #[];
+ Trees ql = #[];
+ for ( Tree b: bl )
+ match b {
+ case bind(`p,`d):
+ pl = pl.append(p);
+ };
+ Trees pvs = #[];
+ for ( Tree g: gl )
+ match g {
+ case bind(`p,`d):
+ ql = ql.append(p);
+ ul = ul.append(d);
+ pvs = pvs.append(pattern_variables(p));
+ };
+ Tree tql = make_tuple(ql);
+ Tree tul = make_tuple(ul);
+ Tree tpl = make_tuple(pl);
+ Trees xl = #[];
+ Trees partl = #[];
+ for ( Tree x: pattern_variables(#<tuple(...pl)>) )
+ if (!pvs.member(x)) {
+ partl = partl.append(#<bind(`x,`x)>);
+ match rename(#<select(`x,from(bind(`tpl,group)),where(true))>) {
+ case select(`hd,`binds,...):
+ xl = xl.append(#<bind(`x,bag(select(`hd,`binds,where(true))))>);
+ }
+ };
+ match rename(#<select(record(...partl),from(bind(`tpl,group)),where(true))>) {
+ case select(`hd,`binds,...):
+ xl = xl.cons(#<bind(partition,bag(select(`hd,`binds,where(true))))>);
+ }
+ tpl = subst(#<any>,#<0>,tpl);
+ ret = #<select(`u,from(bind(tuple(`tql,group),
+ groupBy(select(tuple(`tul,`tpl),from(...bl),where(`c)))),
+ ...xl),where(`h))>;
+ return remove_groupby(ret);
+ case intersect(`x,`y):
+ return remove_groupby(#<select(x,from(bind(x,`x),bind(y,`y)),
+ where(call(eq,x,y)))>);
+ case except(`x,`y):
+ return remove_groupby(#<select(x,from(bind(x,`x)),
+ where(call(not,call(exists,select(y,from(bind(y,`y)),
+ where(call(eq,x,y)))))))>);
+ case member(`x,`y):
+ return remove_groupby(#<call(exists,select(y,from(bind(y,`y)),
+ where(call(eq,y,`x))))>);
+ case call(gen,`min,`max,`size):
+ return #<gen(`(remove_groupby(min)),`(remove_groupby(max)),`(remove_groupby(size)))>;
+ case call(avg,`s):
+ return remove_groupby(#<call(avg_value,call(avg_aggr,`s))>);
+ case call(`f,...al):
+ Tree macro = global_macros.lookup(f.toString());
+ if (macro == null)
+ fail;
+ match macro {
+ case macro(params(...pl),`body):
+ Tree b = rename(remove_groupby(body));
+ if (pl.length() != al.length())
+ fail;
+ for ( ; !pl.is_empty(); pl = pl.tail(), al = al.tail() )
+ b = subst(pl.head(),remove_groupby(al.head()),b);
+ return b;
+ }
+ case call(`f,...al):
+ if (#[cmap,join,mapReduce,mapReduce2,groupBy,orderBy,tuple,bag,list,set].member(f))
+ return remove_groupby(#<`(f.toString())(...al)>);
+ else fail
+ case project(`x,`a):
+ return #<project(`(remove_groupby(x)),`a)>;
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(remove_groupby(a));
+ return #<`f(...bl)>;
+ case `v:
+ if (v.is_variable()) {
+ ret = global_vars.lookup(v.toString());
+ if (ret == null)
+ return v;
+ else if (!v.equals(ret))
+ return remove_groupby(ret);
+ }
+ };
+ return e;
+ }
+
+ private static Tree make_and ( Trees tests ) {
+ if (tests.is_empty())
+ return #<true>;
+ Tree e = tests.head();
+ for ( Tree t: tests.tail() )
+ e = #<call(and,`e,`t)>;
+ return e;
+ }
+
+ private static Trees rename_list ( Trees al ) {
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(rename(a));
+ return bl;
+ }
+
+ /** compile away patterns and rename local variables of an MRQL expression e with unique names */
+ static Tree rename ( Tree e ) {
+ Tree ret = #<error>;
+ match e {
+ case `v:
+ if (!v.is_variable())
+ fail;
+ ret = st.lookup(v.toString());
+ if (ret==null)
+ return v;
+ else return ret;
+ case select(`u,from(...bl),where(`c)):
+ st.begin_scope();
+ Trees binds = #[];
+ Trees tests = #[];
+ for ( Tree b: bl )
+ match b {
+ case bind(`p,`d):
+ Tree x = new_var();
+ binds = binds.append(#<bind(`x,`(rename(d)))>);
+ tests = tests.append(bind_pattern(p,x));
+ };
+ c = make_and(tests.cons(c));
+ ret = #<select(`(rename(u)),
+ from(...binds),
+ where(`(rename(c))))>;
+ st.end_scope();
+ return ret;
+ case lambda(`p,`b):
+ st.begin_scope();
+ Tree nv = new_var();
+ if (!bind_pattern(p,nv).is_empty())
+ error("Lambda patterns must be irrefutable: "+print_query(e));
+ ret = #<lambda(`nv,`(rename(b)))>;
+ st.end_scope();
+ return ret;
+ case function(tuple(...params),`outp,`body):
+ st.begin_scope();
+ Trees ps = #[];
+ Trees vs = #[];
+ for ( Tree p: params )
+ match p {
+ case `bind(`v,`tp):
+ Tree nv = new_var();
+ if (vs.member(v))
+ error("Duplicate function parameters: "+print_query(e));
+ vs = vs.append(v);
+ ps = ps.append(#<`bind(`nv,`tp)>);
+ st.insert(v.toString(),nv);
+ };
+ ret = #<function(tuple(...ps),`outp,`(rename(body)))>;
+ st.end_scope();
+ return ret;
+ case let(`p,`u,`b):
+ Tree ne = rename(u);
+ st.begin_scope();
+ Tree nv = new_var();
+ if (!bind_pattern(p,nv).is_empty())
+ error("Let patterns must be irrefutable: "+print_query(e));
+ ret = #<let(`nv,`ne,`(rename(b)))>;
+ st.end_scope();
+ return ret;
+ case case(`u,...cs):
+ Trees rs = cs.reverse();
+ Tree nu = rename(u);
+ match rs.head() {
+ case case(`p,`b):
+ Trees conds = bind_pattern(p,nu);
+ if (!conds.is_empty())
+ error("Non-exhaustive case "+print_query(p)+" in "+print_query(e));
+ ret = b;
+ };
+ for ( Tree c: rs.tail() )
+ match c {
+ case case(`p,`b):
+ Trees conds = bind_pattern(p,nu);
+ if (!conds.is_empty())
+ ret = #<if(`(make_and(conds)),`b,`ret)>;
+ else error("Unreachable case "+print_query(p)+" in "+print_query(e));
+ };
+ return rename(ret);
+ case project(`u,`a):
+ return #<project(`(rename(u)),`a)>;
+ case bind(`a,`u):
+ return #<bind(`a,`(rename(u)))>;
+ case loop(lambda(tuple(...vs),`b),`s,`n):
+ return #<loop(lambda(tuple(...vs),`(rename(b))),`(rename(s)),`n)>;
+ case `f(...al):
+ Trees bl = rename_list(al);
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ private static Trees has_existential ( Tree e ) {
+ match e {
+ case call(and(`x,`y)):
+ Trees xs = has_existential(x);
+ Trees ys = has_existential(y);
+ return #[call(and(`(xs.head()),`(ys.head())),...(xs.tail()),...(ys.tail()))];
+ case call(exists,select(...)):
+ return #[true,`e];
+ case call(not,call(all,select(...l))):
+ return #[true,call(exists,select(...l))];
+ };
+ return #[`e];
+ }
+
+ /** normalize algebraic expressions to more efficient forms using heuristic rules */
+ public static Tree normalize ( Tree e ) {
+ match e {
+ case select(`u,from(),where(true)):
+ return normalize(#<bag(`u)>);
+ case select(`u,from(),where(`p)):
+ return normalize(#<if(`p,bag(`u),bag())>);
+ case select(`u,from(bind(`v,`d)),where(true)):
+ if (u.equals(v))
+ return normalize(d);
+ else fail
+ case select(`u,from(...bl,bind(`v,select(`iu,from(...ibl),where(`ic))),...al),where(`c)):
+ return normalize(#<select(`u,from(...bl,...ibl,bind(`v,bag(`iu)),...al),
+ where(call(and,`c,`ic)))>);
+ case select(`u,from(...bl,bind(`v,bag(`d)),...al),`c):
+ if (!is_pure(d) && occurences(v,#<f(`c,`u,...al)>) > 1) // duplicated side-effects
+ fail;
+ return normalize(#<select(`(subst(v,d,u)),
+ from(...bl,...(subst_list(v,d,al))),
+ `(subst(v,d,c)))>);
+ case select(`u,from(...bl),where(`c)):
+ Trees es = has_existential(c);
+ if (es.length() <= 1)
+ fail;
+ Trees binds = bl;
+ Trees preds = #[`(es.head())];
+ for ( Tree x: es.tail() )
+ match x {
+ case call(exists,select(`p,from(...bl2),where(`c2))):
+ preds = preds.cons(p).cons(c2);
+ binds = binds.append(bl2);
+ };
+ return normalize(#<select(`u,from(...binds),where(`(make_and(preds))))>);
+ case let_bind(`v,`x,`y):
+ return #<let(`v,`(normalize(x)),`(normalize(y)))>;
+ case call(eq,tuple(...l),`x):
+ Tree pl = #<true>;
+ int i = 0;
+ for ( Tree y: l ) {
+ pl = #<call(and,`pl,call(eq,`y,nth(`x,`i)))>;
+ i++;
+ };
+ return normalize(pl);
+ case call(eq,`x,tuple(...l)):
+ Tree pl = #<true>;
+ int i = 0;
+ for (Tree y: l) {
+ pl = #<call(and,`pl,call(eq,nth(`x,`i),`y))>;
+ i++;
+ };
+ return normalize(pl);
+ case call(and,true,`u): return normalize(u);
+ case call(and,`u,true): return normalize(u);
+ case call(and,false,`u): return #<false>;
+ case call(and,`u,false): return #<false>;
+ case call(or,true,`u): return #<true>;
+ case call(or,`u,true): return #<true>;
+ case call(or,false,`u): return normalize(u);
+ case call(or,`u,false): return normalize(u);
+ case call(not,true): return #<false>;
+ case call(not,false): return #<true>;
+ case if(true,`e1,`e2): return normalize(e1);
+ case if(false,`e1,`e2): return normalize(e2);
+ case nth(tuple(...al),`n):
+ if (!n.is_long())
+ fail;
+ int i = (int)n.longValue();
+ if ( i >= 0 && i < al.length() )
+ return normalize(al.nth(i));
+ case project(record(...bl),`a):
+ for ( Tree b: bl )
+ match b {
+ case bind(`v,`u): if (v.equals(a)) return normalize(u);
+ };
+ error("Wrong projection: "+print_query(e));
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(normalize(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ /** normalize algebraic expressions to more efficient forms using heuristic rules */
+ public static Tree normalize_all ( Tree e ) {
+ Tree ne = normalize(e);
+ if (e.equals(ne))
+ return e;
+ else return normalize(ne);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/ParsedDataSource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/ParsedDataSource.java b/core/src/main/java/org/apache/mrql/ParsedDataSource.java
new file mode 100644
index 0000000..e000405
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/ParsedDataSource.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+
+
+/** A data source for a text HDFS file along with the parser to parse it */
+public class ParsedDataSource extends DataSource {
+ public Class<? extends Parser> parser;
+ public Trees args;
+
+ ParsedDataSource ( int source_num,
+ String path,
+ Class<? extends Parser> parser,
+ Trees args,
+ Configuration conf ) {
+ super(source_num,path,Evaluator.evaluator.parsedInputFormat(),conf);
+ this.parser = parser;
+ this.args = args;
+ }
+
+ ParsedDataSource ( String path,
+ Class<? extends Parser> parser,
+ Trees args,
+ Configuration conf ) {
+ super(-1,path,Evaluator.evaluator.parsedInputFormat(),conf);
+ this.parser = parser;
+ this.args = args;
+ }
+
+ public String toString () {
+ try {
+ String pn = "";
+ for ( String k: DataSource.parserDirectory.keySet() )
+ if (DataSource.parserDirectory.get(k).equals(parser))
+ pn = k;
+ return "Text"+separator+source_num+separator+pn+separator+path
+ +separator+(new Node("args",args)).toString();
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Parser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Parser.java b/core/src/main/java/org/apache/mrql/Parser.java
new file mode 100644
index 0000000..c1728f1
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Parser.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+
+/** for a new text data source, you must implement a new parser */
+interface Parser {
+ public void initialize ( Trees args );
+ public Tree type ();
+ public void open ( String file );
+ public void open ( FSDataInputStream fsin, long start, long end );
+ public String slice ();
+ public Bag parse ( String s );
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Plan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Plan.java b/core/src/main/java/org/apache/mrql/Plan.java
new file mode 100644
index 0000000..c4951bf
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Plan.java
@@ -0,0 +1,362 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.io.*;
+import java.util.Random;
+import java.util.ArrayList;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.SequenceFile.Sorter;
+
+
+/** A physical plan (a superclass for both MapReduce, BSP, and Spark plans) */
+public class Plan {
+ public static Configuration conf;
+ static ArrayList<String> temporary_paths = new ArrayList<String>();
+ private static Random random_generator = new Random();
+ final static int max_input_files = 100;
+
+ /** generate a new path name in HDFS to store intermediate results */
+ public static String new_path ( Configuration conf ) throws IOException {
+ String dir = (Config.local_mode)
+ ? ((Config.tmpDirectory == null) ? "/tmp/mrql" : Config.tmpDirectory)
+ : "mrql";
+ Path p;
+ do {
+ p = new Path(dir+"/mrql"+random_generator.nextInt(1000000));
+ } while (p.getFileSystem(conf).exists(p));
+ String path = p.toString();
+ temporary_paths.add(path);
+ DataSource.dataSourceDirectory.distribute(conf);
+ return path;
+ }
+
+ /** remove all temporary files */
+ public static void clean () throws IOException {
+ for (String p: temporary_paths)
+ try {
+ Path path = new Path(p);
+ path.getFileSystem(conf).delete(path,true);
+ } catch (Exception ex) {
+ FileSystem.getLocal(conf).delete(new Path(p),true);
+ };
+ temporary_paths.clear();
+ DataSource.dataSourceDirectory.clear();
+ }
+
+ /** return the data set size in bytes */
+ public final static long size ( DataSet s ) {
+ return s.size(conf);
+ }
+
+ /** the cache that holds all local data in memory */
+ static Tuple cache;
+
+ /** return the cache element at location loc */
+ public static synchronized MRData getCache ( int loc ) {
+ return cache.get(loc);
+ }
+
+ /** set the cache element at location loc to value and return ret */
+ public static synchronized MRData setCache ( int loc, MRData value, MRData ret ) {
+ if (value instanceof Bag)
+ ((Bag)value).materialize();
+ cache.set(loc,value);
+ return ret;
+ }
+
+ /** put the jar file that contains the compiled MR functional parameters into the TaskTracker classpath */
+ final static void distribute_compiled_arguments ( Configuration conf ) {
+ try {
+ if (!Config.compile_functional_arguments)
+ return;
+ Path local_path = new Path("file://"+Compiler.jar_path);
+ if (Config.spark_mode)
+ conf.set("mrql.jar.path",local_path.toString());
+ else {
+ // distribute the jar file with the compiled arguments to all clients
+ Path hdfs_path = new Path("mrql-tmp/class"+random_generator.nextInt(1000000)+".jar");
+ FileSystem fs = hdfs_path.getFileSystem(conf);
+ fs.copyFromLocalFile(false,true,local_path,hdfs_path);
+ temporary_paths.add(hdfs_path.toString());
+ conf.set("mrql.jar.path",hdfs_path.toString());
+ }
+ } catch (Exception ex) {
+ throw new Error(ex);
+ }
+ }
+
+ /** retrieve the compiled functional argument of code */
+ final static Function functional_argument ( Configuration conf, Tree code ) {
+ Node n = (Node)code;
+ if (n.name().equals("compiled"))
+ try {
+ // if the clent has not received the jar file with the compiled arguments, copy the file from HDFS
+ if (Compiler.jar_path == null) {
+ Path hdfs_path = new Path(conf.get("mrql.jar.path"));
+ String local_path = Compiler.tmp_dir+"/mrql_args_"+random_generator.nextInt(1000000)+".jar";
+ FileSystem fs = hdfs_path.getFileSystem(conf);
+ fs.copyToLocalFile(false,hdfs_path,new Path("file://"+local_path));
+ Compiler.jar_path = local_path;
+ };
+ return Compiler.compiled(conf.getClassLoader(),n.children().nth(0).toString());
+ } catch (Exception ex) {
+ System.err.println("*** Warning: Unable to retrieve the compiled lambda: "+code);
+ return ((Lambda) Interpreter.evalE(n.children().nth(1))).lambda();
+ }
+ else if (code.equals(Interpreter.identity_mapper))
+ return new Function () {
+ public MRData eval ( final MRData x ) { return new Bag(x); }
+ };
+ else return ((Lambda) Interpreter.evalE(code)).lambda();
+ }
+
+ /** comparator for MRData keys */
+ public final static class MRContainerKeyComparator implements RawComparator<MRContainer> {
+ int[] container_size;
+
+ public MRContainerKeyComparator () {
+ container_size = new int[1];
+ }
+
+ final public int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl ) {
+ return MRContainer.compare(x,xs,xl,y,ys,yl,container_size);
+ }
+
+ final public int compare ( MRContainer x, MRContainer y ) {
+ return x.compareTo(y);
+ }
+ }
+
+ /** The source physical operator for binary files */
+ public final static DataSet binarySource ( int source_num, String file ) {
+ return new DataSet(new BinaryDataSource(source_num,file,conf),0,0);
+ }
+
+ /** The source physical operator for binary files */
+ public final static DataSet binarySource ( String file ) {
+ return new DataSet(new BinaryDataSource(-1,file,conf),0,0);
+ }
+
+ /** splits the range min..max into multiple ranges, one for each mapper */
+ public final static DataSet generator ( int source_num, long min, long max, long split_length ) throws Exception {
+ if (min > max)
+ throw new Error("Wrong range: "+min+"..."+max);
+ if (split_length < 1)
+ if (Config.bsp_mode)
+ split_length = (max-min)/Config.nodes+1;
+ else split_length = Config.range_split_size;
+ DataSet ds = new DataSet(0,0);
+ long i = min;
+ while (i+split_length <= max) {
+ String file = new_path(conf);
+ Path path = new Path(file);
+ SequenceFile.Writer writer
+ = SequenceFile.createWriter(path.getFileSystem(conf),conf,path,
+ MRContainer.class,MRContainer.class,
+ SequenceFile.CompressionType.NONE);
+ writer.append(new MRContainer(new MR_long(i)),
+ new MRContainer(new Tuple(new MR_long(i),new MR_long(split_length))));
+ writer.close();
+ ds.source.add(new GeneratorDataSource(source_num,file,conf));
+ i += split_length;
+ };
+ if (i <= max) {
+ String file = new_path(conf);
+ Path path = new Path(file);
+ SequenceFile.Writer writer
+ = SequenceFile.createWriter(path.getFileSystem(conf),conf,path,
+ MRContainer.class,MRContainer.class,
+ SequenceFile.CompressionType.NONE);
+ writer.append(new MRContainer(new MR_long(i)),
+ new MRContainer(new Tuple(new MR_long(i),new MR_long(max-i+1))));
+ writer.close();
+ ds.source.add(new GeneratorDataSource(source_num,file,conf));
+ };
+ return ds;
+ }
+
+ /** splits the range min..max into multiple ranges, one for each mapper */
+ public final static DataSet generator ( long min, long max, long split_length ) throws Exception {
+ return generator(-1,min,max,split_length);
+ }
+
+ /** The source physical operator for parsing text files */
+ public final static DataSet parsedSource ( int source_num, Class<? extends Parser> parser, String file, Trees args ) {
+ return new DataSet(new ParsedDataSource(source_num,file,parser,args,conf),0,0);
+ }
+
+ /** The source physical operator for parsing text files */
+ public final static DataSet parsedSource ( Class<? extends Parser> parser, String file, Trees args ) {
+ return new DataSet(new ParsedDataSource(file,parser,args,conf),0,0);
+ }
+
+ /** merge the sorted files of the data source */
+ public final static Bag merge ( final DataSource s ) throws Exception {
+ Path path = new Path(s.path);
+ final FileSystem fs = path.getFileSystem(conf);
+ final FileStatus[] ds
+ = fs.listStatus(path,
+ new PathFilter () {
+ public boolean accept ( Path path ) {
+ return !path.getName().startsWith("_");
+ }
+ });
+ int dl = ds.length;
+ if (dl == 0)
+ return new Bag();
+ Path[] paths = new Path[dl];
+ for ( int i = 0; i < dl; i++ )
+ paths[i] = ds[i].getPath();
+ if (dl > Config.max_merged_streams) {
+ if (Config.trace)
+ System.err.println("*** Merging "+dl+" files");
+ Path out_path = new Path(new_path(conf));
+ SequenceFile.Sorter sorter
+ = new SequenceFile.Sorter(fs,new MRContainerKeyComparator(),
+ MRContainer.class,MRContainer.class,conf);
+ sorter.merge(paths,out_path);
+ paths = new Path[1];
+ paths[0] = out_path;
+ };
+ final int n = paths.length;
+ SequenceFile.Reader[] sreaders = new SequenceFile.Reader[n];
+ for ( int i = 0; i < n; i++ )
+ sreaders[i] = new SequenceFile.Reader(fs,paths[i],conf);
+ final SequenceFile.Reader[] readers = sreaders;
+ final MRContainer[] keys_ = new MRContainer[n];
+ final MRContainer[] values_ = new MRContainer[n];
+ for ( int i = 0; i < n; i++ ) {
+ keys_[i] = new MRContainer();
+ values_[i] = new MRContainer();
+ };
+ return new Bag(new BagIterator () {
+ int min = 0;
+ boolean first = true;
+ final MRContainer[] keys = keys_;
+ final MRContainer[] values = values_;
+ final MRContainer key = new MRContainer();
+ final MRContainer value = new MRContainer();
+ public boolean hasNext () {
+ if (first)
+ try {
+ first = false;
+ for ( int i = 0; i < n; i++ )
+ if (readers[i].next(key,value)) {
+ keys[i].set(key.data());
+ values[i].set(value.data());
+ } else {
+ keys[i] = null;
+ readers[i].close();
+ }
+ } catch (IOException e) {
+ throw new Error("Cannot merge values from an intermediate result");
+ };
+ min = -1;
+ for ( int i = 0; i < n; i++ )
+ if (keys[i] != null && min < 0)
+ min = i;
+ else if (keys[i] != null && keys[i].compareTo(keys[min]) < 0)
+ min = i;
+ return min >= 0;
+ }
+ public MRData next () {
+ try {
+ MRData res = values[min].data();
+ if (readers[min].next(key,value)) {
+ keys[min].set(key.data());
+ values[min].set(value.data());
+ } else {
+ keys[min] = null;
+ readers[min].close();
+ };
+ return res;
+ } catch (IOException e) {
+ throw new Error("Cannot merge values from an intermediate result");
+ }
+ }
+ });
+ }
+
+ /** The collect physical operator */
+ public final static Bag collect ( final DataSet x, boolean strip ) throws Exception {
+ return Evaluator.evaluator.parsedInputFormat().newInstance().collect(x,strip);
+ }
+
+ /** The collect physical operator */
+ public final static Bag collect ( final DataSet x ) throws Exception {
+ return collect(x,true);
+ }
+
+ /** the DataSet union physical operator */
+ public final static DataSet merge ( final DataSet x, final DataSet y ) throws IOException {
+ DataSet res = x;
+ res.source.addAll(y.source);
+ return res;
+ }
+
+ final static MR_long counter_key = new MR_long(0);
+ final static MRContainer counter_container = new MRContainer(counter_key);
+ final static MRContainer value_container = new MRContainer(new MR_int(0));
+
+ /** The cache operator that dumps a bag into an HDFS file */
+ public final static DataSet fileCache ( Bag s ) throws IOException {
+ String newpath = new_path(conf);
+ Path path = new Path(newpath);
+ FileSystem fs = path.getFileSystem(conf);
+ SequenceFile.Writer writer
+ = new SequenceFile.Writer(fs,conf,path,
+ MRContainer.class,MRContainer.class);
+ long i = 0;
+ for ( MRData e: s ) {
+ counter_key.set(i++);
+ value_container.set(e);
+ writer.append(counter_container,value_container);
+ };
+ writer.close();
+ return new DataSet(new BinaryDataSource(0,newpath,conf),0,0);
+ }
+
+ /** for dumped data to a file, return the MRQL type of the data */
+ public final static Tree get_type ( String file ) {
+ try {
+ Path path = new Path(file);
+ FileSystem fs = path.getFileSystem(conf);
+ BufferedReader ftp = new BufferedReader(new InputStreamReader(fs.open(path.suffix(".type"))));
+ String s[] = ftp.readLine().split("@");
+ ftp.close();
+ if (s.length != 2 )
+ return null;
+ if (!s[0].equals("2"))
+ throw new Error("The binary file has been created in java mode and cannot be read in hadoop mode");
+ return Tree.parse(s[1]);
+ } catch (Exception e) {
+ return null;
+ }
+ }
+
+ /** create a new PrintStream from the file */
+ final static PrintStream print_stream ( String file ) throws Exception {
+ Path path = new Path(file);
+ FileSystem fs = path.getFileSystem(conf);
+ return new PrintStream(fs.create(path));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/PlanGeneration.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/PlanGeneration.gen b/core/src/main/java/org/apache/mrql/PlanGeneration.gen
new file mode 100644
index 0000000..59b8cb8
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/PlanGeneration.gen
@@ -0,0 +1,716 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.*;
+import java.io.*;
+
+
+/** Generate a physical plan from an algebraic expression */
+final public class PlanGeneration extends AlgebraicOptimization {
+
+ /** extract the combiner from the reducer in a MapReduce plan */
+ static class Aggregates {
+ public static Trees maps = #[];
+ public static Trees combines = #[];
+ public static Trees reduces = #[];
+ public static boolean can_use_combiner = true;
+ private static SymbolTable st = new SymbolTable();
+
+ private static void clear () {
+ maps = #[];
+ combines = #[];
+ reduces = #[];
+ can_use_combiner = true;
+ }
+
+ /** extends the combines, maps, and reduces lists with new entries */
+ private static int union_aggegates ( Tree reduce, Tree map, Tree combine ) {
+ Tree m = simplify_all(map);
+ Tree c = simplify_all(combine);
+ Tree rd = simplify_all(reduce);
+ int i = 0;
+ for ( Trees r = reduces; !r.is_empty(); r = r.tail(), i++ )
+ if (alpha_equivalent(rd,r.head()))
+ return i;
+ maps = maps.append(m);
+ reduces = reduces.append(rd);
+ combines = combines.append(subst(#<-1>,#<`i>,c));
+ return i;
+ }
+
+ /** Generate the MR combiner from the MR reducer.
+ * Find the aggregation calls (eg, call(avg,_)) in the reducer
+ * @param e the body of the reducer
+ * @param map the map function
+ * @param mvar the variable of the map function
+ * @param rvar the variable of the reducer function
+ * @return the combiner, but also sets the combines, maps, and reduces lists
+ */
+ private static Tree derive_combiner ( Tree e, Tree map, Tree mvar, Tree rvar ) {
+ Tree gvar = #<nth(`rvar,1)>;
+ match e {
+ case call(`f,`u):
+ for ( Tree monoid: monoids ) // system & user-defined aggregations
+ match monoid {
+ case `nm(`mtp,`plus,`zero,`unit):
+ if (nm.equals(f.toString())) {
+ match TypeInference.type_inference2(u) {
+ case `S(`tp):
+ if (TypeInference.unify(tp,mtp) == null)
+ continue;
+ case _: throw new Error("Unexpected aggregation: "+e);
+ };
+ match u {
+ case cmap(`m,`v):
+ if (!v.equals(gvar) || occurences(rvar,m) > 0)
+ fail;
+ Tree ev = new_var();
+ Tree nv = new_var();
+ Tree mv = new_var();
+ int i = union_aggegates(e,
+ #<aggregate(lambda(`ev,apply(`plus,tuple(nth(`ev,0),apply(`unit,nth(`ev,1))))),
+ `zero,cmap(`m,cmap(lambda(x,bag(nth(x,1))),apply(`map,`mvar))))>,
+ #<aggregate(lambda(`nv,apply(`plus,tuple(nth(`nv,0),
+ nth(nth(`nv,1),-1)))),
+ `zero,`gvar)>);
+ return simplify_all(#<aggregate(lambda(`mv,apply(`plus,tuple(nth(`mv,0),
+ nth(nth(`mv,1),`i)))),
+ `zero,`gvar)>);
+ case `v:
+ if (!v.equals(gvar))
+ fail;
+ Tree ev = new_var();
+ Tree nv = new_var();
+ Tree mv = new_var();
+ int i = union_aggegates(e,
+ #<aggregate(lambda(`ev,apply(`plus,tuple(nth(`ev,0),apply(`unit,nth(`ev,1))))),
+ `zero,cmap(lambda(x,bag(nth(x,1))),apply(`map,`mvar)))>,
+ #<aggregate(lambda(`nv,apply(`plus,tuple(nth(`nv,0),nth(nth(`nv,1),-1)))),
+ `zero,`gvar)>);
+ return simplify_all(#<aggregate(lambda(`mv,apply(`plus,tuple(nth(`mv,0),
+ nth(nth(`mv,1),`i)))),
+ `zero,`gvar)>);
+ }
+ }
+ };
+ fail
+ case nth(`v,0):
+ if (v.is_variable())
+ return e;
+ else fail
+ case `f(...al):
+ Trees rs = #[];
+ for ( Tree a: al )
+ rs = rs.append(derive_combiner(a,map,mvar,rvar));
+ return #<`f(...rs)>;
+ };
+ if (#<nth(`e,1)>.equals(gvar))
+ Aggregates.can_use_combiner = false;
+ return e;
+ }
+ }
+
+ /** how many times e accesses the bag x? if it's more than one, it can't be streamed */
+ private static int number_of_accesses ( Tree x, Tree e ) {
+ if (e.equals(x))
+ return 1;
+ match e {
+ case cmap(`m,`s):
+ return number_of_accesses(x,m)*10+number_of_accesses(x,s);
+ case map(`m,`s):
+ return number_of_accesses(x,m)*10+number_of_accesses(x,s);
+ case filter(`p,`m,`s):
+ return number_of_accesses(x,p)*10+number_of_accesses(x,m)*10+number_of_accesses(x,s);
+ case `f(...r):
+ int i = 0;
+ for ( Tree z: r )
+ i += number_of_accesses(x,z);
+ return i;
+ };
+ return 0;
+ }
+
+ /** can we process the second arg of the MapReducer reducer (a bag) as a stream? */
+ public static boolean streamed_MapReduce_reducer ( Tree x ) {
+ match x {
+ case lambda(`v,`b):
+ return number_of_accesses(#<nth(`v,1)>,b) <= 1;
+ case compiled(_,lambda(`v,`b)):
+ return number_of_accesses(#<nth(`v,1)>,b) <= 1;
+ };
+ return false;
+ }
+
+ /** can we process the first arg of the MapReducer2 reducer (a bag) as a stream? */
+ public static boolean streamed_MapReduce2_reducer ( Tree x ) {
+ match x {
+ case lambda(`v,`b):
+ return number_of_accesses(#<nth(`v,0)>,b) <= 1;
+ case compiled(_,lambda(`v,`b)):
+ return number_of_accesses(#<nth(`v,0)>,b) <= 1;
+ };
+ return false;
+ }
+
+ /** true if e returns a dataset stored in HDFS */
+ public static boolean is_dataset_expr ( Tree e ) {
+ match TypeInference.type_inference2(e) {
+ case `T(_):
+ if (is_persistent_collection(T))
+ return true;
+ };
+ return false;
+ }
+
+ /** compile an algebraic form to a algebraic plan
+ * @param e the algebraic form
+ * @return the algebraic plan
+ */
+ public static Tree makePlan ( Tree e ) {
+ match e {
+ // combine groupBy with Join (experimental)
+ case mapReduce(lambda(`vm,`bm),lambda(`vr,`br),`s,`o):
+ if (!Config.use_combiner || !is_dataset_expr(s))
+ fail;
+ Tree splan = makePlan(s);
+ match splan {
+ case MapReduce2(lambda(`mvx,bag(tuple(`kx,`mx))),
+ lambda(`mvy,bag(tuple(`ky,`my))),
+ lambda(`v,cmap(lambda(`x,cmap(lambda(`y,bag(tuple(tuple(`gx,`gy),`b))),
+ nth(`vx,1))),
+ nth(`vy,0))),
+ `X,`Y,`o2):
+ if (!vx.equals(v) || !vy.equals(v) || !mx.equals(mvx) || !my.equals(mvy))
+ fail;
+ Tree gxx = gx;
+ Tree gyy = gy;
+ if (free_variables(gx,#[]).equals(#[`y]) && free_variables(gy,#[]).equals(#[`x])) {
+ gxx = gy;
+ gyy = gx;
+ } else if (!free_variables(gx,#[]).equals(#[`x]) || !free_variables(gy,#[]).equals(#[`y]))
+ fail;
+ Aggregates.clear();
+ Tree nv = new_var();
+ match TypeInference.type_inference(bm) {
+ case `S(`tp):
+ if (!is_collection(S))
+ fail;
+ type_env.insert(nv.toString(),tp);
+ };
+ Tree rd = Aggregates.derive_combiner(br,#<lambda(`vm,`bm)>,vm,vr);
+ if (Aggregates.reduces.is_empty())
+ fail;
+ Tree vr2 = new_var();
+ Tree m = simplify_all(#<lambda(`vm,cmap(lambda(`nv,bag(tuple(nth(`nv,0),
+ tuple(...(Aggregates.maps))))),`bm))>);
+ Tree c = subst(vr,vr2,#<bag(tuple(...(Aggregates.combines)))>);
+ c = simplify_all(#<lambda(`vr2,`c)>);
+ Tree r = simplify_all(#<lambda(`vr,`rd)>);
+ Tree mtp = TypeInference.type_inference(#<bag(tuple(...(Aggregates.maps)))>);
+ Tree rtp = #<tuple(`(TypeInference.type_inference(#<nth(`vr,0)>)),`mtp)>;
+ type_env.insert(vr.toString(),rtp);
+ type_env.insert(vr2.toString(),rtp);
+ TypeInference.type_inference(m);
+ TypeInference.type_inference(c);
+ TypeInference.type_inference(r);
+ Tree combiner = makePlan(c);
+ Tree reducer = makePlan(r);
+ Tree nm = simplify_all(subst(x,#<nth(`vx,0)>,
+ subst(y,#<nth(`vy,1)>,
+ #<cmap(lambda(x,bag(nth(x,1))),
+ cmap(`m,bag(tuple(tuple(`gx,`gy),`b))))>)));
+ type_env.insert(v.toString(),TypeInference.type_inference(#<tuple(`mvx,`mvy)>));
+ nm = makePlan(nm);
+ return #<GroupByJoin(lambda(`mvx,`kx),
+ lambda(`mvy,`ky),
+ lambda(`x,`gxx),
+ lambda(`y,`gyy),
+ lambda(`v,`nm),
+ `combiner,
+ `reducer,
+ `X,`Y,`o)>;
+ };
+ fail
+ // extract the mapReduce combiner
+ case mapReduce(lambda(`vm,`bm),lambda(`vr,`br),`s,`o):
+ if (!Config.use_combiner || !is_dataset_expr(s))
+ fail;
+ Tree splan = makePlan(s);
+ Aggregates.clear();
+ Tree nv = new_var();
+ match TypeInference.type_inference(bm) {
+ case `S(`tp):
+ if (!is_collection(S))
+ fail;
+ type_env.insert(nv.toString(),tp);
+ };
+ Tree rd = Aggregates.derive_combiner(br,#<lambda(`vm,`bm)>,vm,vr);
+ if (!Aggregates.can_use_combiner || Aggregates.reduces.is_empty())
+ fail;
+ Tree vr2 = new_var();
+ Tree m = simplify_all(#<lambda(`vm,cmap(lambda(`nv,bag(tuple(nth(`nv,0),
+ tuple(...(Aggregates.maps))))),`bm))>);
+ Tree c = subst(vr,vr2,#<bag(tuple(...(Aggregates.combines)))>);
+ c = simplify_all(#<lambda(`vr2,`c)>);
+ Tree r = simplify_all(#<lambda(`vr,`rd)>);
+ Tree mtp = TypeInference.type_inference(#<bag(tuple(...(Aggregates.maps)))>);
+ Tree rtp = #<tuple(`(TypeInference.type_inference(#<nth(`vr,0)>)),`mtp)>;
+ type_env.insert(vr.toString(),rtp);
+ type_env.insert(vr2.toString(),rtp);
+ TypeInference.type_inference(m);
+ TypeInference.type_inference(c);
+ TypeInference.type_inference(r);
+ Tree combiner = makePlan(c);
+ Tree reducer = makePlan(r);
+ match splan {
+ // if the MapCombineReduce input is a join, push the combiner to the join
+ case MapReduce2(`mx,`my,lambda(`rv,`rb),`x,`y,`o2):
+ Tree nr = makePlan(simplify_all(#<lambda(`rv,cmap(`m,`rb))>));
+ return #<MapReduce(lambda(`vm,bag(`vm)),`reducer,
+ MapCombineReduce2(`mx,`my,`combiner,`nr,`x,`y,`o2),`o)>;
+ case `input:
+ return #<MapCombineReduce(`(makePlan(m)),`combiner,`reducer,`input,`o)>;
+ };
+ fail
+ case mapReduce(`m,`r,`s,`o):
+ if (is_dataset_expr(s))
+ return #<MapReduce(`(makePlan(m)),
+ `(makePlan(r)),
+ `(makePlan(s)),`o)>;
+ else fail
+ case mapReduce2(`mx,`my,`r,`x,`y,`o):
+ if (is_dataset_expr(x) && is_dataset_expr(y) && streamed_MapReduce2_reducer(r))
+ return #<MapReduce2(`(makePlan(mx)),
+ `(makePlan(my)),
+ `(makePlan(r)),
+ `(makePlan(x)),
+ `(makePlan(y)),`o)>;
+ else fail
+ case mapReduce2(`mx,`my,lambda(`v,`b),`x,`y,`o):
+ if (!is_dataset_expr(x) || !is_dataset_expr(y))
+ fail;
+ // if the join reducer is not streaming, switch the inputs
+ Tree nv = new_var();
+ Tree nr = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
+ subst(#<nth(`v,1)>,#<nth(`nv,0)>,b));
+ nr = #<lambda(`nv,`nr)>;
+ type_env.insert(nv.toString(),TypeInference.type_inference(#<tuple(nth(`v,1),nth(`v,0))>));
+ return #<MapReduce2(`(makePlan(my)),
+ `(makePlan(mx)),
+ `(makePlan(nr)),
+ `(makePlan(y)),
+ `(makePlan(x)),`o)>;
+ case crossProduct(`mx,`my,`r,`x,`y):
+ if (is_dataset_expr(x) && is_dataset_expr(y))
+ return #<CrossProduct(`(makePlan(mx)),
+ `(makePlan(my)),
+ `(makePlan(r)),
+ `(makePlan(x)),
+ `(makePlan(y)))>;
+ else fail
+ case cmap(`m,`s):
+ if (is_dataset_expr(s))
+ return #<cMap(`(makePlan(m)),
+ `(makePlan(s)))>;
+ else fail
+ case cmap(lambda(`v,if(`p,`T(`u),`S())),`s):
+ if (false && is_collection(T) && is_collection(S))
+ return makePlan(#<filter(lambda(`v,`p),lambda(`v,`u),`s)>);
+ else fail
+ case call(source,binary,`file,`tp):
+ return #<BinarySource(`file,`tp)>;
+ case call(source,gen,`f,`len,`ulen):
+ return #<SequenceSource(`(makePlan(f)),`(makePlan(len)),
+ `(makePlan(ulen)))>;
+ case call(source,`parser,`file,...args):
+ Trees el = #[];
+ for ( Tree a: args )
+ el = el.append(makePlan(a));
+ return #<ParsedSource(`parser,`(makePlan(file)),...el)>;
+ case type(`x): return e;
+ case gen(`min,`max,`size):
+ return #<Generator(`(makePlan(min)),`(makePlan(max)),`(makePlan(size)))>;
+ case repeat(lambda(`v,`b),`s,`n):
+ if (!is_dataset_expr(s))
+ fail;
+ repeat_variables = repeat_variables.cons(v);
+ return #<Repeat(lambda(`v,`(makePlan(b))),`(makePlan(s)),
+ `(makePlan(n)))>;
+ case repeat(lambda(`v,`b),`s):
+ if (!is_dataset_expr(s))
+ fail;
+ repeat_variables = repeat_variables.cons(v);
+ return #<Repeat(lambda(`v,`(makePlan(b))),`(makePlan(s)),`(Integer.MAX_VALUE))>;
+ case closure(lambda(`v,`b),`s,`n):
+ if (!is_dataset_expr(s))
+ fail;
+ repeat_variables = repeat_variables.cons(v);
+ return #<Closure(lambda(`v,`(makePlan(b))),`(makePlan(s)),
+ `(makePlan(n)))>;
+ case closure(lambda(`v,`b),`s):
+ if (!is_dataset_expr(s))
+ fail;
+ repeat_variables = repeat_variables.cons(v);
+ return #<Closure(lambda(`v,`(makePlan(b))),`(makePlan(s)),`(Integer.MAX_VALUE))>;
+ case loop(lambda(tuple(...vs),`b),`s,`n):
+ repeat_variables = repeat_variables.append(vs);
+ return #<Loop(lambda(tuple(...vs),`(makePlan(b))),`(makePlan(s)),`(makePlan(n)))>;
+ case record(...bl):
+ Trees el = #[];
+ for ( Tree b: bl )
+ match b {
+ case bind(_,`c):
+ el = el.append(c);
+ };
+ return makePlan(#<tuple(...el)>);
+ case project(`x,`a):
+ Tree tp = TypeInference.type_inference(x);
+ match tp {
+ case XML:
+ return makePlan(#<call(XMLchildren,`(new StringLeaf(a.toString())),`x)>);
+ case `S(XML):
+ if (is_collection(S))
+ return makePlan(#<call(XMLchildren,`(new StringLeaf(a.toString())),`x)>);
+ };
+ match TypeInference.expand(tp) {
+ case record(...bl):
+ int i = 0;
+ for ( Tree b: bl )
+ match b {
+ case bind(`c,_):
+ if (a.equals(c))
+ return makePlan(#<nth(`x,`i)>);
+ else fail
+ case _: i++;
+ };
+ case union(...tl):
+ for ( Tree t: tl )
+ match TypeInference.expand(t) {
+ case _(record(...bl)):
+ int i = 0;
+ for ( Tree b: bl )
+ match b {
+ case bind(`c,_):
+ if (a.equals(c))
+ return makePlan(#<nth(union_value(`x),`i)>);
+ else fail
+ case _: i++;
+ };
+ case _(bag(tuple(string,`tv))):
+ return #<map_index(`(makePlan(#<union_value(`x)>)),
+ `(new StringLeaf(a.toString())))>;
+ case `tt: error("wrong projection: "+e+" ("+tt+")");
+ };
+ case `S(`ttp):
+ if (!is_collection(S))
+ fail;
+ match TypeInference.expand(ttp) {
+ case tuple(string,`tv):
+ return #<map_index(`(makePlan(x)),
+ `(new StringLeaf(a.toString())))>;
+ case record(...bl):
+ Tree nv = new_var();
+ int i = 0;
+ for ( Tree b: bl )
+ match b {
+ case bind(`c,_):
+ if (a.equals(c))
+ return makePlan(#<cmap(lambda(`nv,nth(`nv,`i)),`x)>);
+ else fail
+ case _: i++;
+ };
+ case union(...tl):
+ Tree nv = new_var();
+ int j = 0;
+ for ( Tree t: tl ) {
+ match t {
+ case `c(record(...bl)):
+ int i = 0;
+ for ( Tree b: bl )
+ match b {
+ case bind(`w,_):
+ if (a.equals(w))
+ return makePlan(#<cmap(lambda(`nv,if(call(eq,union_tag(`nv),`j),
+ bag(nth(union_value(`nv),`i)),
+ bag())),`x)>);
+ else fail
+ case _: i++;
+ };
+ case `c(bag(tuple(string,`tv))):
+ return makePlan(#<cmap(lambda(`nv,if(call(eq,union_tag(`nv),`j),
+ bag(map_index(union_value(`nv),
+ `(new StringLeaf(a.toString())))),
+ bag())),`x)>);
+ case `tt: error("wrong projection: "+e+" ("+tt+")");
+ };
+ j++;
+ }
+ };
+ case `t: error("wrong projection: "+e+" ("+t+")");
+ };
+ case typed(`x,`tp):
+ if (tp.is_variable() && !tp.equals(#<string>) && MRContainer.type_code(tp.toString()) >= 0)
+ return makePlan(#<call(coerce,`x,
+ `(MRContainer.type_code(tp.toString())))>);
+ else fail
+ case index(`x,`n):
+ match TypeInference.type_inference2(x) {
+ case bag(tuple(`kt,`vt)):
+ if (TypeInference.type_inference2(n).equals(kt))
+ return #<map_index(`(makePlan(x)),
+ `(makePlan(n)))>;
+ case Bag(tuple(`kt,`vt)):
+ if (TypeInference.type_inference2(n).equals(kt))
+ return #<map_index(`(makePlan(x)),
+ `(makePlan(n)))>;
+ case union(...tl):
+ for ( Tree t: tl )
+ match TypeInference.expand(t) {
+ case _(bag(tuple(`kt,`vt))):
+ if (TypeInference.type_inference2(n).equals(kt))
+ return #<map_index(`(makePlan(#<union_value(`x)>)),
+ `(makePlan(n)))>;
+ case _(list(`tp)):
+ return #<index(`(makePlan(#<union_value(`x)>)),
+ `(makePlan(n)))>;
+ case `tt: error("wrong indexing: "+e+" ("+tt+")");
+ };
+ };
+ return #<index(`(makePlan(x)),
+ `(makePlan(n)))>;
+ case call(count,mapReduce(`m,lambda(`vr,`br),`X,`o)):
+ Tree nv = new_var();
+ type_env.insert(nv.toString(),TypeInference.type_inference(vr));
+ Tree nr = simplify(#<lambda(`nv,bag(call(count,`(subst(vr,nv,br)))))>);
+ Tree plan = #<call(sum,mapReduce(`m,`nr,`X,false))>;
+ return makePlan(plan);
+ case call(`f,`u):
+ for ( Tree monoid: monoids )
+ match monoid {
+ case `aggr(`mtp,`plus,`zero,`unit):
+ if (aggr.equals(f.toString())) {
+ Tree plan = makePlan(u);
+ Tree nx = new_var();
+ Tree np = new_var();
+ Tree na = new_var();
+ Tree tp = TypeInference.type_inference(e);
+ type_env.insert(np.toString(),#<tuple(`tp,`tp)>);
+ match TypeInference.type_inference(u) {
+ case `T(`t):
+ if (TypeInference.unify(mtp,t) == null)
+ continue;
+ type_env.insert(na.toString(),#<tuple(`tp,`t)>);
+ type_env.insert(nx.toString(),t);
+ };
+ plus = makePlan(simplify_all(#<lambda(`np,apply(`plus,`np))>));
+ Tree acc = makePlan(simplify_all(#<lambda(`na,apply(`plus,tuple(nth(`na,0),
+ apply(`unit,nth(`na,1)))))>));
+ zero = makePlan((f.equals(#<avg>)) ? zero : #<typed(`zero,`tp)>);
+ match plan {
+ case MapCombineReduce(`m,`c,`r,`s,_):
+ plan = #<MapAggregateReduce(`m,`r,`acc,`zero,`s,false)>;
+ case MapReduce(`m,`r,`s,_):
+ plan = #<MapAggregateReduce(`m,`r,`acc,`zero,`s,false)>;
+ case MapReduce2(`mx,`my,`r,`x,`y,_):
+ plan = #<MapAggregateReduce2(`mx,`my,`r,`acc,`zero,`x,`y,false)>;
+ case MapJoin(`kx,`ky,`r,`x,`y):
+ plan = #<MapAggregateJoin(`kx,`ky,`r,`acc,`zero,`x,`y)>;
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ plan = #<CrossAggregateProduct(`mx,`my,`r,`acc,`zero,`x,`y)>;
+ case cMap(`m,`s):
+ plan = #<AggregateMap(`m,`acc,`zero,`s)>;
+ case _:
+ if (is_dataset_expr(u))
+ plan = #<AggregateMap(lambda(`nx,bag(`nx)),`acc,`zero,`plan)>;
+ else return #<aggregate(`acc,`zero,`plan)>;
+ };
+ if (is_dataset_expr(u))
+ return #<Aggregate(`plus,`zero,`plan)>;
+ else return #<aggregate(`plus,`zero,`plan)>;
+ }
+ };
+ fail
+ case call(plus,`x,`y):
+ if (!is_dataset_expr(x) || !is_dataset_expr(y))
+ fail;
+ return #<Merge(`(makePlan(x)),`(makePlan(y)))>;
+ case call(`f,...el):
+ if (!f.is_variable())
+ fail;
+ Tree ret = data_constructors.lookup(f.toString());
+ if (ret != null)
+ match ret {
+ case `v(`n,`tp):
+ Tree p = (el.length()==1)
+ ? makePlan(el.head())
+ : makePlan(#<tuple(...el)>);
+ return #<tagged_union(`n,`p)>;
+ };
+ ret = global_type_env.lookup(f.toString());
+ if (ret != null)
+ match ret {
+ case arrow(_,_):
+ Trees args = #[];
+ for ( Tree a: el )
+ args = args.append(makePlan(a));
+ return #<apply(`f,tuple(...args))>;
+ };
+ Trees tps = #[];
+ for ( Tree a: el )
+ tps = tps.append(TypeInference.type_inference(a));
+ int i = ClassImporter.find_method_number(f.toString(),tps);
+ if (i < 0)
+ error("Method "+f+tps+" has no implementation");
+ Trees sig = ClassImporter.signature(i);
+ Trees args = #[];
+ for ( int j = 0; j < el.length(); j++ ) {
+ Tree b = sig.nth(j+1);
+ if (f.equals(#<coerce>) || b.equals(tps.nth(j)) || !b.is_variable()
+ || b.equals(#<union>) || MRContainer.type_code(b.toString()) < 0)
+ args = args.append(makePlan(el.nth(j)));
+ else args = args.append(makePlan(#<call(coerce,`(el.nth(j)),
+ `(MRContainer.type_code(b.toString())))>));
+ };
+ return #<callM(`f,`i,...args)>;
+ case let(`v,`u,`body):
+ if (true)
+ fail; // disable
+ body = makePlan(body);
+ match TypeInference.type_inference(u) {
+ case `S(_):
+ // if a variable bound to a collection is used more than once in the body,
+ // materialize the collection in memory
+ if (is_collection(S) && occurences(v,body) > 1)
+ body = #<let(`v,`(makePlan(#<call(materialize,`v)>)),`body)>;
+ };
+ return #<let(`v,`(makePlan(u)),`body)>;
+ case function(tuple(...params),`outp,`body):
+ boolean is_dataset = false;
+ for ( Tree p: params )
+ match p {
+ case dataset(`v,`tp):
+ is_dataset = true;
+ };
+ body = makePlan(body);
+ return #<function(tuple(...params),`outp,`body)>;
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(makePlan(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ /** convert an algebraic plan to a physical plan
+ * @param plan algebraic plan
+ * @return the physical plan
+ */
+ public static Tree physical_plan ( Tree plan ) {
+ match plan {
+ case MapReduce2(`mx,`my,`r,`x,`y,`o):
+ return physical_plan(#<MapAggregateReduce2(`mx,`my,`r,null,null,`x,`y,`o)>);
+ // convert a reduce-side join to a fragment-replicate join, if either of the join
+ // inputs is small to fit in memory (dataset sizes are extracted from file statistics)
+ case MapAggregateReduce2(`mx,`my,
+ lambda(`v,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nx,1))),nth(`ny,0))),
+ null,null,`x,`y,false):
+ if (Config.noMapJoin)
+ fail;
+ if (!nx.equals(v) || !ny.equals(v) || occurences(v,b) > 0)
+ fail;
+ Tree X = new_var();
+ Tree Y = new_var();
+ Tree nv = new_var();
+ Tree L1 = simplify_all(#<lambda(`nv,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nv,1))),bag(nth(`nv,0))))>);
+ nv = new_var();
+ Tree L2 = simplify_all(#<lambda(`nv,cmap(lambda(`vx,cmap(lambda(`vy,`b),bag(nth(`nv,0)))),nth(`nv,1)))>);
+ nv = new_var();
+ Tree L3 = #<lambda(`nv,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nv,0))),nth(`nv,1)))>;
+ Tree cond1 = makePlan(#<call(leq,dataset_size(`Y),`(Config.mapjoin_size))>);
+ Tree cond2 = makePlan(#<call(leq,dataset_size(`X),`(Config.mapjoin_size))>);
+ Tree cond3 = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
+ return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
+ If(`cond1,
+ MapJoin(`mx,`my,`L1,`X,`Y),
+ If(`cond2,
+ MapJoin(`my,`mx,`L2,`Y,`X),
+ If(`cond3,
+ MapAggregateReduce2(`my,`mx,`L3,null,null,`Y,`X,false),
+ MapAggregateReduce2(`mx,`my,
+ lambda(`v,cmap(lambda(`vx,cmap(lambda(`vy,`b),nth(`nx,1))),
+ nth(`ny,0))),
+ null,null,`X,`Y,false))))))>;
+ case MapAggregateReduce2(`mx,`my,lambda(`v,`b),null,null,`x,`y,false):
+ if (Config.noMapJoin)
+ fail;
+ Tree X = new_var();
+ Tree Y = new_var();
+ Tree nv = new_var();
+ Tree L1 = simplify_all(subst(#<nth(`v,0)>,#<bag(nth(`nv,0))>,
+ subst(#<nth(`v,1)>,#<nth(`nv,1)>,
+ #<lambda(`nv,`b)>)));
+ nv = new_var();
+ Tree L2 = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
+ subst(#<nth(`v,1)>,#<nth(`nv,0)>,
+ #<lambda(`nv,`b)>));
+ Tree cond1 = makePlan(#<call(leq,dataset_size(`Y),`(Config.mapjoin_size))>);
+ Tree cond2 = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
+ return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
+ If(`cond1,
+ MapJoin(`mx,`my,`L1,`X,`Y),
+ If(`cond2,
+ MapAggregateReduce2(`my,`mx,`L2,null,null,`Y,`X,false),
+ MapAggregateReduce2(`mx,`my,lambda(`v,`b),null,null,`X,`Y,false)))))>;
+ case MapAggregateReduce2(`mx,`my,lambda(`v,`b),`acc,`zero,`x,`y,false):
+ Tree X = new_var();
+ Tree Y = new_var();
+ Tree nv = new_var();
+ Tree nr = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
+ subst(#<nth(`v,1)>,#<nth(`nv,0)>,b));
+ if (!streamed_MapReduce2_reducer(#<lambda(`nv,`nr)>))
+ fail;
+ Tree cond = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
+ return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
+ If(`cond,
+ MapAggregateReduce2(`my,`mx,lambda(`nv,`nr),`acc,`zero,`Y,`X,false),
+ MapAggregateReduce2(`mx,`my,lambda(`v,`b),`acc,`zero,`X,`Y,false))))>;
+ case CrossProduct(`mx,`my,`r,`x,`y):
+ return physical_plan(#<CrossAggregateProduct(`mx,`my,`r,null,null,`x,`y)>);
+ case CrossAggregateProduct(`mx,`my,lambda(`v,`b),`acc,`zero,`x,`y):
+ Tree X = new_var();
+ Tree Y = new_var();
+ Tree nv = new_var();
+ Tree nr = subst(#<nth(`v,0)>,#<nth(`nv,1)>,
+ subst(#<nth(`v,1)>,#<nth(`nv,0)>,b));
+ Tree cond = makePlan(#<call(lt,dataset_size(`X),dataset_size(`Y))>);
+ return #<Let(`X,`(physical_plan(x)),Let(`Y,`(physical_plan(y)),
+ If(`cond,
+ CrossAggregateProduct(`my,`mx,lambda(`nv,`nr),`acc,`zero,`Y,`X),
+ CrossAggregateProduct(`mx,`my,lambda(`v,`b),`acc,`zero,`X,`Y))))>;
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(physical_plan(a));
+ return #<`f(...bl)>;
+ };
+ return plan;
+ }
+}
[17/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/XPathParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/XPathParser.java b/core/src/main/java/org/apache/mrql/XPathParser.java
new file mode 100644
index 0000000..faded33
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/XPathParser.java
@@ -0,0 +1,692 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.Stack;
+import org.xml.sax.*;
+import org.xml.sax.helpers.DefaultHandler;
+import org.xml.sax.helpers.AttributesImpl;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+
+/** Compiles XPath queries to SAX pipelines */
+final public class XPathParser {
+ public MRDataHandler dataConstructor;
+ public XPathHandler handler;
+ final static PrintStream out = System.out;
+ static int cached_events = 0;
+
+ /** a SAX handler to form XPath pipelines */
+ abstract class XPathHandler extends DefaultHandler {
+ XPathHandler next; // the next handler in the pipeline
+
+ public XPathHandler ( XPathHandler next ) {
+ this.next = next;
+ }
+
+ public void startDocument () throws SAXException {
+ next.startDocument();
+ }
+
+ public void endDocument () throws SAXException {
+ next.endDocument();
+ }
+
+ abstract public void startElement ( String uri, String name, String tag, Attributes atts ) throws SAXException;
+
+ abstract public void endElement ( String uri, String name, String tag ) throws SAXException;
+
+ abstract public void characters ( char text[], int start, int length ) throws SAXException;
+
+ /** start a new predicate */
+ public void startPredicate ( int pred ) {
+ next.startPredicate(pred);
+ }
+
+ /** the end of a predicate */
+ public void endPredicate ( int pred ) {
+ next.endPredicate(pred);
+ }
+
+ public void releasePredicate ( int pred ) { // set the predicate outcome to true
+ next.releasePredicate(pred);
+ }
+ }
+
+ /** The end of the pipeline: Print the SAX stream to the output */
+ final class Print extends XPathHandler {
+
+ public Print () {
+ super(null);
+ }
+
+ public void startDocument () {}
+
+ public void endDocument () { out.println(); }
+
+ public void startElement ( String uri, String name, String tag, Attributes atts ) {
+ out.append("<").append(tag);
+ if (atts != null)
+ for (int i = 0; i < atts.getLength(); i++)
+ out.append(" ").append(atts.getQName(i))
+ .append("=\"").append(atts.getValue(i)).append("\"");
+ out.append(">");
+ }
+
+ public void endElement ( String uri, String name, String tag ) {
+ out.append("</").append(tag).append(">");
+ }
+
+ public void characters ( char text[], int start, int length ) {
+ for (int i = 0; i < length; i++)
+ out.append(text[start+i]);
+ }
+
+ public String toString () { return "print()"; }
+ }
+
+ /** A growable buffer for storing events as byte sequences */
+ final class Cache {
+ final static int buffer_size_increment = 1000;
+ public byte[] wrapped_buffer;
+ public ByteBuffer byteBuffer;
+
+ public Cache () {
+ wrapped_buffer = new byte[buffer_size_increment];
+ byteBuffer = ByteBuffer.wrap(wrapped_buffer);
+ }
+
+ private void grow ( int len ) {
+ len /= java.lang.Byte.SIZE;
+ while (len+byteBuffer.position() > byteBuffer.limit()) {
+ int pos = byteBuffer.position();
+ byte[] nb = new byte[wrapped_buffer.length+buffer_size_increment];
+ for (int i = 0; i < wrapped_buffer.length; i++)
+ nb[i] = wrapped_buffer[i];
+ wrapped_buffer = nb;
+ byteBuffer = ByteBuffer.wrap(nb);
+ byteBuffer.position(pos);
+ }
+ }
+
+ public void putByte ( byte n ) {
+ grow(8);
+ byteBuffer.put(n);
+ }
+
+ public void putShort ( short n ) {
+ grow(Short.SIZE);
+ byteBuffer.putShort(n);
+ }
+
+ public void putInt ( int n ) {
+ grow(Integer.SIZE);
+ byteBuffer.putInt(n);
+ }
+
+ public void putChars ( char text[], int start, int len ) {
+ grow(len*Character.SIZE+Integer.SIZE);
+ byteBuffer.putInt(len);
+ for (int i = 0; i < len; i++)
+ byteBuffer.putChar(text[start+i]);
+ }
+
+ public void putString ( String s ) {
+ grow(s.length()*Character.SIZE+Short.SIZE);
+ int len = s.length();
+ byteBuffer.putShort((short) len);
+ for (int i = 0; i < len; i++)
+ byteBuffer.putChar(s.charAt(i));
+ }
+
+ public byte getByte () { return byteBuffer.get(); }
+
+ public short getShort () { return byteBuffer.getShort(); }
+
+ public int getInt () { return byteBuffer.getInt(); }
+
+ public char[] getChars () {
+ int len = byteBuffer.getInt();
+ char[] buf = new char[len];
+ for (int i = 0; i < len; i++)
+ buf[i] = byteBuffer.getChar();
+ return buf;
+ }
+
+ public String getString () {
+ int len = byteBuffer.getShort();
+ char[] buf = new char[len];
+ for (int i = 0; i < len; i++)
+ buf[i] = byteBuffer.getChar();
+ return new String(buf);
+ }
+
+ public void cacheStartElement ( String uri, String name, String tag, Attributes atts ) {
+ cached_events++;
+ putByte((byte)0);
+ putString(uri);
+ putString(name);
+ putString(tag);
+ if (atts != null) {
+ putShort((short) atts.getLength());
+ for (int i = 0; i < atts.getLength(); i++) {
+ putString(atts.getQName(i));
+ putString(atts.getValue(i));
+ }
+ } else putShort((short) 0);
+ }
+
+ public void cacheEndElement ( String uri, String name, String tag ) {
+ cached_events++;
+ putByte((byte)1);
+ putString(uri);
+ putString(name);
+ putString(tag);
+ }
+
+ public void cacheCharacters ( char text[], int start, int length ) {
+ cached_events++;
+ putByte((byte)2);
+ putChars(text,start,length);
+ }
+
+ public void print () {
+ System.out.println(byteBuffer);
+ dump(new Print());
+ }
+
+ public void append ( Cache cache ) {
+ grow(cache.byteBuffer.position());
+ byte[] b = cache.byteBuffer.array();
+ byteBuffer.put(b,0,cache.byteBuffer.position());
+ cache.byteBuffer.clear();
+ }
+
+ /** regenerate the stream from buffer */
+ public void dump ( XPathHandler next ) {
+ int last = byteBuffer.position();
+ byteBuffer.position(0);
+ while (byteBuffer.position() < last)
+ try {
+ switch (getByte()) {
+ case 0:
+ String uri = getString();
+ String name = getString();
+ String tag = getString();
+ AttributesImpl atts = new AttributesImpl();
+ int len = getShort();
+ for (int i = 0; i < len; i++)
+ atts.addAttribute("","",getString(),"",getString());
+ next.startElement(uri,name,tag,atts);
+ break;
+ case 1:
+ next.endElement(getString(),getString(),getString());
+ break;
+ case 2:
+ char[] text = getChars();
+ next.characters(text,0,text.length);
+ }
+ } catch (SAXException e) {
+ throw new Error(e);
+ };
+ byteBuffer.clear();
+ }
+ }
+
+ /** Remove the start/end/releasePredicate events by storing some events in a buffer, when necessary */
+ final class Materialize extends XPathHandler {
+ final static int max_num_of_nested_predicates = 100;
+ final public Cache cache; // nested suspended events from predicates whose outcome is unknown
+ final int[] ids; // the ids of the predicates with suspended output
+ final int[] positions; // position of predicate events in the buffer
+ final boolean[] released; // true if the associated predicate is true
+ int top; // top of the stacks
+
+ public Materialize ( XPathHandler next ) {
+ super(next);
+ cache = new Cache();
+ ids = new int[max_num_of_nested_predicates];
+ positions = new int[max_num_of_nested_predicates];
+ released = new boolean[max_num_of_nested_predicates];
+ top = 0;
+ }
+
+ public void startElement ( String uri, String name, String tag, Attributes atts ) throws SAXException {
+ if (top > 0)
+ cache.cacheStartElement(uri,name,tag,atts);
+ else next.startElement(uri,name,tag,atts);
+ }
+
+ public void endElement ( String uri, String name, String tag ) throws SAXException {
+ if (top > 0)
+ cache.cacheEndElement(uri,name,tag);
+ else next.endElement(uri,name,tag);
+ }
+
+ public void characters ( char text[], int start, int length ) throws SAXException {
+ if (top > 0)
+ cache.cacheCharacters(text,start,length);
+ else next.characters(text,start,length);
+ }
+
+ public void startPredicate ( int pred ) {
+ if (top >= ids.length)
+ throw new Error("too many nested predicates");
+ positions[top] = cache.byteBuffer.position();
+ ids[top] = pred;
+ released[top++] = false;
+ }
+
+ public void endPredicate ( int pred ) {
+ if (top > 0 && ids[top-1] == pred)
+ cache.byteBuffer.position(positions[--top]).mark().reset();
+ }
+
+ public void releasePredicate ( int pred ) {
+ boolean flush = true;
+ for (int i = 0; i < top; i++)
+ if (ids[i] == pred)
+ released[i] = true;
+ else flush &= released[i];
+ if (top > 0 && flush) {
+ cache.dump(next);
+ top = 0;
+ }
+ }
+
+ public String toString () { return "materialize("+next+")"; }
+ }
+
+ /** return the children of the current nodes that have the given tagname */
+ final class Child extends XPathHandler {
+ final String tagname; // the tagname of the child
+ boolean keep; // are we keeping or skipping events?
+ short level; // the depth level of the current element
+
+ public Child ( String tagname, XPathHandler next ) {
+ super(next);
+ this.tagname = tagname;
+ keep = false;
+ level = 0;
+ }
+
+ public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
+ if (level++ == 1)
+ keep = tagname.equals("*") || tagname.equals(qn);
+ if (keep)
+ next.startElement(nm,ln,qn,a);
+ }
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {
+ if (keep)
+ next.endElement(nm,ln,qn);
+ if (--level == 1)
+ keep = false;
+ }
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ if (keep)
+ next.characters(text,start,length);
+ }
+
+ public String toString () { return "child("+tagname+","+next+")"; }
+ }
+
+ /** return the attribute value of the current nodes that have the given attribute name */
+ final class Attribute extends XPathHandler {
+ final String attributename;
+ short level; // the depth level of the current element
+
+ public Attribute ( String attribute_name, XPathHandler next ) {
+ super(next);
+ attributename = attribute_name;
+ level = 0;
+ }
+
+ public void startElement ( String nm, String ln, String qn, Attributes as ) throws SAXException {
+ if (level++ == 0)
+ for ( int i = 0; i < as.getLength(); i++ )
+ if (attributename.equals("*") || attributename.equals(as.getQName(i))) {
+ char[] s = as.getValue(i).toCharArray();
+ next.characters(s,0,s.length);
+ }
+ }
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {
+ --level;
+ }
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ }
+
+ public String toString () { return "attribute("+attributename+","+next+")"; }
+ }
+
+ /** Return the descendants of the current nodes that have the given tagname.
+ * To handle nested elements with the same tagname, use Descendant
+ */
+ final class SimpleDescendant extends XPathHandler {
+ final String tagname; // the tagname of the descendant
+ boolean keep; // are we keeping or skipping events?
+
+ public SimpleDescendant ( String tagname, XPathHandler next ) {
+ super(next);
+ this.tagname = tagname;
+ keep = false;
+ }
+
+ public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
+ if (!keep)
+ keep = tagname.equals(qn);
+ if (keep)
+ next.startElement(nm,ln,qn,a);
+ }
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {
+ if (keep) {
+ next.endElement(nm,ln,qn);
+ keep = !tagname.equals(qn);
+ }
+ }
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ if (keep)
+ next.characters(text,start,length);
+ }
+
+ public String toString () { return "simple_descendant("+tagname+","+next+")"; }
+ }
+
+ /** As efficient as SimpleDescendant when there are no nested elements with the same tagname.
+ * It caches only the inner nested subelements with the same tagname.
+ */
+ final class Descendant extends XPathHandler {
+ final static int max_nested_level = 100;
+ final String tagname; // the tagname of the descendant
+ int level; // # of nested elements with the same tagname
+ final Cache[] cache; // cache[i] caches elements of level i-1
+
+ public Descendant ( String tagname, XPathHandler next ) {
+ super(next);
+ this.tagname = tagname;
+ cache = new Cache[max_nested_level]; // to be created lazily
+ level = 0;
+ }
+
+ public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
+ if (tagname.equals(qn)) {
+ if (level > 0 && cache[level-1] == null)
+ cache[level-1] = new Cache();
+ level++;
+ };
+ for (int i = 1; i < level; i++)
+ cache[i-1].cacheStartElement(nm,ln,qn,a);
+ if (level > 0)
+ next.startElement(nm,ln,qn,a);
+ }
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {
+ if (level > 0)
+ next.endElement(nm,ln,qn);
+ for (int i = 1; i < level; i++)
+ cache[i-1].cacheEndElement(nm,ln,qn);
+ if (tagname.equals(qn)) {
+ level--;
+ if (level == 0 && cache[0] != null)
+ cache[0].dump(next);
+ else if (level > 0 && cache[level] != null)
+ cache[level-1].append(cache[level]);
+ }
+ }
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ for (int i = 1; i < level; i++)
+ cache[i-1].cacheCharacters(text,start,length);
+ if (level > 0)
+ next.characters(text,start,length);
+ }
+
+ public String toString () { return "descendant("+tagname+","+next+")"; }
+ }
+
+ /** propagates all input signals to both next and condition streams but wraps each
+ * top-level element in the next stream with start/end Condition signals
+ */
+ final class Predicate extends XPathHandler {
+ int level;
+ final XPathHandler condition; // false, if stream is empty
+ final int predicate_id; // the id of the predicate
+
+ public Predicate ( int predicate_id, XPathHandler condition, XPathHandler next ) {
+ super(next);
+ this.condition = condition;
+ this.predicate_id = predicate_id;
+ level = 0;
+ }
+
+ public void startPredicate ( int pred ) {
+ next.startPredicate(pred);
+ condition.startPredicate(pred);
+ }
+
+ public void endPredicate ( int pred ) {
+ next.endPredicate(pred);
+ condition.endPredicate(pred);
+ }
+
+ public void releasePredicate ( int pred ) {
+ next.releasePredicate(pred);
+ condition.releasePredicate(pred);
+ }
+
+ public void startDocument () throws SAXException {
+ next.startDocument();
+ condition.startDocument();
+ }
+
+ public void endDocument () throws SAXException {
+ next.endDocument();
+ condition.endDocument();
+ }
+
+ public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
+ if (level++ == 0)
+ next.startPredicate(predicate_id);
+ next.startElement(nm,ln,qn,a);
+ condition.startElement(nm,ln,qn,a);
+ }
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {
+ next.endElement(nm,ln,qn);
+ condition.endElement(nm,ln,qn);
+ if (--level == 0)
+ next.endPredicate(predicate_id);
+ }
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ next.characters(text,start,length);
+ condition.characters(text,start,length);
+ }
+
+ public String toString () { return "predicate("+predicate_id+","+condition+","+next+")"; }
+ }
+
+ /** generate a releasePredicate signal if the content of the input node is equal to text */
+ final class Equals extends XPathHandler {
+ final static int max_depth_of_nested_predicates = 100;
+ final String value; // the value to be tested for equality
+ final int predicate_id; // the id of the predicate
+ final int[] preds;
+ int top;
+ boolean suspended;
+
+ public Equals ( int predicate_id, String value, XPathHandler next ) {
+ super(next);
+ this.value = value;
+ this.predicate_id = predicate_id;
+ preds = new int[max_depth_of_nested_predicates];
+ top = 0;
+ suspended = false;
+ }
+
+ private boolean compare ( char[] text, int start, int length, String value ) {
+ if (length != value.length())
+ return false;
+ for (int i = 0; i < length; i++)
+ if (text[i+start] != value.charAt(i))
+ return false;
+ return true;
+ }
+
+ public void startPredicate ( int pred ) {
+ preds[top++] = pred;
+ }
+
+ public void endPredicate ( int pred ) {
+ suspended = false;
+ for (int i = 0; i < top; i++)
+ if (preds[i] == pred) {
+ preds[i] = preds[--top];
+ return;
+ }
+ }
+
+ public void releasePredicate ( int pred ) {
+ if (top == 1 && suspended)
+ next.releasePredicate(predicate_id);
+ endPredicate(pred);
+ }
+
+ public void startDocument () throws SAXException {}
+
+ public void endDocument () throws SAXException {}
+
+ public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {}
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {}
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ if (compare(text,start,length,value))
+ if (top == 0)
+ next.releasePredicate(predicate_id);
+ else suspended = true;
+ }
+
+ public String toString () { return "equals("+predicate_id+","+value+","+next+")"; }
+ }
+
+ /** Converts the SAX data stream to MRData */
+ final class MRDataHandler extends XPathHandler {
+ Stack<Union> stack = new Stack<Union>();
+ Bag children;
+
+ public MRDataHandler () throws Exception {
+ super(null);
+ stack.clear();
+ Tuple t = new Tuple(3);
+ children = new Bag();
+ t.set(2,children);
+ stack.push(new Union((byte)0,t));
+ }
+
+ public void start () throws Exception {
+ stack.clear();
+ Tuple t = new Tuple(3);
+ children = new Bag();
+ t.set(2,children);
+ stack.push(new Union((byte)0,t));
+ }
+
+ public Bag value () {
+ if (stack.size() != 1)
+ return null;
+ else return ((Bag)((Tuple)stack.peek().value()).get(2));
+ }
+
+ public void startDocument () throws SAXException {}
+
+ public void endDocument () throws SAXException {}
+
+ public void startElement ( String nm, String ln, String qn, Attributes as ) throws SAXException {
+ children = new Bag();
+ Bag attributes = new Bag();
+ for ( int i = 0; i < as.getLength(); i++ )
+ attributes.add(new Tuple(new MR_string(as.getQName(i)),new MR_string(as.getValue(i))));
+ Tuple t = new Tuple(3);
+ t.set(0,new MR_string(qn));
+ t.set(1,attributes);
+ t.set(2,children);
+ stack.push(new Union((byte)0,t));
+ }
+
+ public void endElement ( String nm, String ln, String qn ) throws SAXException {
+ if (stack.empty())
+ throw new SAXException("Ill-formed XML elements: "+qn);
+ Union v = stack.pop();
+ if (!((MR_string)((Tuple)v.value()).get(0)).get().equals(qn))
+ throw new SAXException("Unmatched tags in XML element: "+qn);
+ children = (Bag)((Tuple)stack.peek().value()).get(2);
+ children.add(v);
+ }
+
+ public void characters ( char[] text, int start, int length ) throws SAXException {
+ String s = new String(text,start,length);
+ if (s.startsWith("{{") && s.endsWith("}}"))
+ children.add(new MR_variable(Integer.parseInt(s.substring(2,s.length()-2))));
+ else children.add(new Union((byte)1,new MR_string(s)));
+ }
+
+ public String toString () { return "MRDataHandler()"; }
+ }
+
+ public XPathParser ( Tree xpath ) throws Exception {
+ dataConstructor = new MRDataHandler();
+ handler = compile_xpath(xpath,new Materialize(dataConstructor),0);
+ }
+
+ public XPathHandler compile_xpath ( Tree xpath, XPathHandler next, int cn ) throws Exception {
+ if (xpath.is_variable() && xpath.toString().equals("dot"))
+ return next;
+ if (!xpath.is_node())
+ throw new Error("Unrecognized xpath query: "+xpath);
+ Node n = (Node) xpath;
+ if (n.name().equals("child")) {
+ String tag = n.children().head().toString();
+ XPathHandler c = (tag.charAt(0) == '@')
+ ? new Attribute(tag.substring(1),next)
+ : new Child(tag,next);
+ return compile_xpath(n.children().tail().head(),c,cn);
+ } else if (n.name().equals("descendant")) {
+ XPathHandler c = new Descendant(n.children().head().toString(),next);
+ return compile_xpath(n.children().tail().head(),c,cn);
+ } else if (n.name().equals("eq")) {
+ Tree value = n.children().tail().head();
+ XPathHandler c = new Equals(cn,(value.is_string())
+ ? ((StringLeaf)value).value()
+ : value.toString(),
+ next);
+ return compile_xpath(n.children().head(),c,cn);
+ } else if (n.name().equals("predicate")) {
+ XPathHandler c = new Predicate(cn+1,compile_xpath(n.children().head(),next,cn+1),next);
+ return compile_xpath(n.children().tail().head(),c,cn);
+ };
+ throw new Error("Unrecognized xpath query: "+xpath);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/mrql.cgen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/mrql.cgen b/core/src/main/java/org/apache/mrql/mrql.cgen
new file mode 100644
index 0000000..9b8dbe4
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/mrql.cgen
@@ -0,0 +1,365 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java_cup.runtime.*;
+import org.apache.mrql.gen.*;
+
+parser code {:
+
+ static int[] tokens = {
+ sym.error, sym.IF, sym.THEN, sym.ELSE, sym.SELECT, sym.FROM, sym.HAVING,
+ sym.LB, sym.RB, sym.LP, sym.RP, sym.LSB, sym.RSB,
+ sym.PLUS, sym.MINUS, sym.TIMES, sym.DIV, sym.MOD, sym.EQ,
+ sym.NEQ, sym.LT, sym.LEQ, sym.GT, sym.GEQ, sym.SHARP, sym.AS,
+ sym.AND, sym.OR, sym.NOT, sym.UNION,
+ sym.INTERSECT, sym.EXCEPT, sym.EXISTS, sym.IN, sym.DOT, sym.COLON, sym.COMMA,
+ sym.SEMI, sym.ASSIGN, sym.WHERE, sym.ORDER, sym.GROUP, sym.BY, sym.ASCENDING,
+ sym.DESCENDING, sym.FUNCTION, sym.DISTINCT, sym.BSLASH, sym.SOME, sym.ALL,
+ sym.SOME, sym.ALL, sym.GTR, sym.SEP, sym.STORE, sym.DUMP, sym.TYPE, sym.DATA, sym.REPEAT,
+ sym.STEP, sym.LIMIT, sym.LET, sym.ATSYM, sym.EXCLAMATION,
+ sym.Variable, sym.Integer, sym.Double, sym.String, sym.Decimal,
+ sym.START_TEMPLATE, sym.END_TEMPLATE, sym.TEXT
+ };
+
+ static String[] token_names = {
+ "error", "if", "then", "else", "select", "from", "having",
+ "[", "]", "(", ")", "{", "}",
+ "+", "-", "*", "/", "mod", "=",
+ "<>", "<", "<=", ">", ">=", "#", "as",
+ "and", "or", "not", "union",
+ "intersect", "except", "exists", "in", ".", ":", ",",
+ ";", ":=", "where", "order", "group", "by", "ascending",
+ "descending", "function", "distinct", "\\", "some", "all",
+ "some", "all", ">", "|", "store", "dump", "type", "data", "repeat",
+ "step", "limit", "let", "@", "!",
+ "Variable", "Integer", "Double", "String", "Decimal",
+ "[|", "|]", "Text"
+ };
+
+ public static String print ( Symbol s ) {
+ for (int i=0; i<tokens.length; i++)
+ if (tokens[i]==s.sym)
+ { String res = token_names[i] + " ";
+ if (s.value==null)
+ return res;
+ else if (s.value instanceof Integer)
+ return res + ((Integer) s.value).intValue();
+ else if (s.value instanceof Float)
+ return res + ((Float) s.value).floatValue();
+ else if (s.value instanceof String)
+ return res + (String) s.value;
+ }
+ return "?";
+ }
+
+ static {
+ Tree.parsed = true;
+ }
+
+ public int line_pos () {
+ return ((MRQLLex)getScanner()).line_pos();
+ }
+
+ public int char_pos () {
+ return ((MRQLLex)getScanner()).char_pos();
+ }
+
+ public void syntax_error ( Symbol token ) {
+ System.err.println("*** Syntax Error: " + print(token) + " (line: " + line_pos() + ", position: " + char_pos() + ")");
+ if (Config.testing)
+ throw new Error("Syntax Error");
+ }
+
+:};
+
+/* Terminals (tokens returned by the scanner). */
+terminal IF, THEN, ELSE, SELECT, FROM, HAVING, LB, RB, LP, RP, LSB, RSB, LDOT, SHARP,
+ PLUS, MINUS, TIMES, DIV, MOD, EQ, NEQ, LT, LEQ, GT, GEQ, AND, OR, NOT, AS,
+ UNION, INTERSECT, EXCEPT, EXISTS, IN, COMMA, DOT, COLON, ASSIGN, SEMI, WHERE,
+ ORDER, GROUP, BY, ASCENDING, DESCENDING, UMINUS, FUNCTION, DISTINCT, BSLASH,
+ SOME, ALL, GTR, SEP, STORE, TYPE, DATA, CASE, ATSYM, XPATH, REPEAT, STEP, LIMIT,
+ LET, IMPORT, PARSER, AGGREGATION, INCLUDE, EXCLAMATION, MACRO, DUMP;
+
+terminal String Variable;
+terminal Long Integer;
+terminal Double Double;
+terminal String String;
+terminal Double Decimal;
+terminal String START_TEMPLATE;
+terminal String END_TEMPLATE;
+terminal String TEXT;
+
+non terminal prog;
+non terminal Tree item, expr, var, const, mrql, pattern, opt_where, opt_orderby,
+ mode, opt_groupby, opt_having, opt_distinct, type, xpath, xname,
+ xpred, opt_limit, unit;
+non terminal Trees expl, name_binds, pat_list, pat_binds,
+ binds, order_binds, groupby_binds, typel, type_binds,
+ data_binds, cases, fnl, template, template_pat, fnc_params, var_list;
+
+precedence nonassoc LDOT, ASSIGN, LIMIT;
+precedence nonassoc ELSE, COLON;
+precedence nonassoc ORDER, GROUP, HAVING, WHERE;
+precedence left INTERSECT, UNION, EXCEPT, IN;
+precedence nonassoc ASCENDING, DESCENDING;
+precedence nonassoc COMMA, LP;
+precedence right OR;
+precedence right AND;
+precedence nonassoc NOT;
+precedence nonassoc EQ, LT, GT, LEQ, GEQ, NEQ;
+precedence left PLUS, MINUS;
+precedence left TIMES, DIV, MOD;
+precedence nonassoc DOT, SHARP, LB, AS;
+precedence nonassoc UMINUS;
+
+start with prog;
+
+prog ::= item:i SEMI {: Translator.top_level(i); :}
+ | prog item:i SEMI {: Translator.top_level(i); :}
+ ;
+item ::= expr:e {: RESULT = #<expression(`e)>; :}
+ | var:v EQ expr:e {: RESULT = #<assign(`v,`e)>; :}
+ | STORE var:v ASSIGN expr:e {: RESULT = #<store(`v,`e)>; :}
+ | STORE String:s FROM expr:e {: RESULT = #<dump(`(new StringLeaf(s)),`e)>; :}
+ | DUMP String:s FROM expr:e {: RESULT = #<dump_text(`(new StringLeaf(s)),`e)>; :}
+ | TYPE var:v EQ type:t {: RESULT = #<typedef(`v,`t)>; :}
+ | DATA var:v EQ data_binds:nl {: RESULT = #<datadef(`v,union(...nl))>; :}
+ | FUNCTION var:f LP
+ fnc_params:p RP COLON type:tp
+ LSB expr:e RSB {: RESULT = #<functiondef(`f,params(...p),`tp,`e)>; :}
+ | FUNCTION var:f LP RP COLON type:tp
+ LSB expr:e RSB {: RESULT = #<functiondef(`f,params(),`tp,`e)>; :}
+ | MACRO var:f LP var_list:vl RP
+ LSB expr:e RSB {: RESULT = #<macrodef(`f,params(...vl),`e)>; :}
+ | IMPORT fnl:l FROM String:c {: RESULT = #<import(`c,...l)>; :}
+ | IMPORT String:c {: RESULT = #<import(`c)>; :}
+ | PARSER var:c EQ String:p {: RESULT = #<parser(`c,`p)>; :}
+ | INCLUDE String:c {: RESULT = #<include(`c)>; :}
+ | AGGREGATION var:n LP expr:a COMMA expr:z unit:u RP COLON type:tp
+ {: RESULT = #<aggregation(`n,`tp,`a,`z,`u)>; :}
+ | error {: :}
+ ;
+unit ::= COMMA expr:u {: RESULT = u; :}
+ | {: RESULT = #<lambda(x,x)>; :}
+ ;
+fnl ::= Variable:v {: RESULT = #[`v]; :}
+ | String:v {: RESULT = #[`v]; :}
+ | Variable:v COMMA fnl:l {: RESULT = #[`v,...l]; :}
+ | String:v COMMA fnl:l {: RESULT = #[`v,...l]; :}
+ ;
+fnc_params ::= fnc_params:nl
+ COMMA var:v COLON type:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
+ | var:v COLON type:e {: RESULT = #[bind(`v,`e)]; :}
+ ;
+var ::= Variable:v {: RESULT = #<`v>; :}
+ ;
+var_list ::= var:v {: RESULT = #[`v]; :}
+ | var_list:el COMMA var:v {: RESULT = el.append(v); :}
+ ;
+const ::= String:s {: RESULT = new StringLeaf(s); :}
+ | Integer:n {: RESULT = new LongLeaf(n.longValue()); :}
+ | Double:n {: RESULT = new DoubleLeaf((double)n.doubleValue()); :}
+ | Decimal:n {: RESULT = new DoubleLeaf((double)n.doubleValue()); :}
+ ;
+expr ::= mrql:e {: RESULT = e; :}
+ | var:v {: RESULT = v; :}
+ | IF expr:p THEN expr:e1 ELSE expr:e2 {: RESULT = #<if(`p,`e1,`e2)>; :}
+ | expr:e1 PLUS expr:e2 {: RESULT = #<call(plus,`e1,`e2)>; :}
+ | expr:e1 MINUS expr:e2 {: RESULT = #<call(minus,`e1,`e2)>; :}
+ | expr:e1 TIMES expr:e2 {: RESULT = #<call(times,`e1,`e2)>; :}
+ | expr:e1 DIV expr:e2 {: RESULT = #<call(div,`e1,`e2)>; :}
+ | expr:e1 MOD expr:e2 {: RESULT = #<call(mod,`e1,`e2)>; :}
+ | expr:e1 EQ expr:e2 {: RESULT = #<call(eq,`e1,`e2)>; :}
+ | expr:e1 NEQ expr:e2 {: RESULT = #<call(neq,`e1,`e2)>; :}
+ | expr:e1 LT expr:e2 {: RESULT = #<call(lt,`e1,`e2)>; :}
+ | expr:e1 LEQ expr:e2 {: RESULT = #<call(leq,`e1,`e2)>; :}
+ | expr:e1 GT expr:e2 {: RESULT = #<call(gt,`e1,`e2)>; :}
+ | expr:e1 GEQ expr:e2 {: RESULT = #<call(geq,`e1,`e2)>; :}
+ | expr:e1 AND expr:e2 {: RESULT = #<call(and,`e1,`e2)>; :}
+ | expr:e1 OR expr:e2 {: RESULT = #<call(or,`e1,`e2)>; :}
+ | expr:e1 UNION expr:e2 {: RESULT = #<call(plus,`e1,`e2)>; :}
+ | expr:e1 INTERSECT expr:e2 {: RESULT = #<intersect(`e1,`e2)>; :}
+ | expr:e1 EXCEPT expr:e2 {: RESULT = #<except(`e1,`e2)>; :}
+ | expr:e DOT var:v {: RESULT = #<project(`e,`v)>; :}
+ | expr:e DOT TIMES {: RESULT = #<call(XMLchildren,`(new StringLeaf("*")),`e)>; :}
+ | expr:e DOT ATSYM Variable:v {: RESULT = #<call(XMLattribute,`(new StringLeaf(v)),`e)>; :}
+ | expr:e DOT ATSYM TIMES {: RESULT = #<call(XMLattributes,`(new StringLeaf("*")),`e)>; :}
+ | expr:e SHARP Integer:n {: RESULT = #<nth(`e,`(new LongLeaf(n.longValue())))>; :}
+ | expr:e LB expr:i RB {: RESULT = #<index(`e,`i)>; :}
+ | expr:e LB expr:i COLON expr:j RB {: RESULT = #<range(`e,`i,`j)>; :}
+ | var:v LP RP {: RESULT = #<call(`v)>; :}
+ | var:v LP expl:el RP {: RESULT = #<call(`v,...el)>; :}
+ | LP RP {: RESULT = #<tuple()>; :}
+ | LP expl:el RP {: RESULT = (el.length()==1)?el.head():#<tuple(...el)>; :}
+ | LSB RSB {: RESULT = #<bag()>; :}
+ | LSB expl:el RSB {: RESULT = #<bag(...el)>; :}
+ | LB RB {: RESULT = #<list()>; :}
+ | LB expl:el RB {: RESULT = #<list(...el)>; :}
+ | LT {: MRQLLex.record_begin(); :}
+ name_binds:nl GTR {: MRQLLex.record_end(); RESULT = #<record(...nl)>; :}
+ | PLUS expr:e {: RESULT = e; :} %prec UMINUS
+ | MINUS expr:e {: RESULT = #<call(minus,`e)>; :} %prec UMINUS
+ | NOT expr:e {: RESULT = #<call(not,`e)>; :}
+ | EXISTS expr:e {: RESULT = #<call(exists,`e)>; :}
+ | CASE expr:e LSB cases:cs RSB {: RESULT = #<case(`e,...cs)>; :}
+ | BSLASH LP fnc_params:p RP
+ COLON type:tp DOT expr:e {: RESULT = #<function(tuple(...p),`tp,`e)>; :} %prec LDOT
+ | expr:e AS type:tp {: RESULT = #<typed(`e,`tp)>; :}
+ | LET pattern:p EQ expr:e IN expr:b {: RESULT = #<let_bind(`p,`e,`b)>; :}
+ | expr:i DOT DOT expr:j {: RESULT = #<range(`i,`j)>; :}
+ | expr:i DOT DOT DOT expr:j {: RESULT = #<gen(`i,`j,-1)>; :}
+ | XPATH LP xpath:x RP {: RESULT = #<xpath(`x)>; :}
+ | TYPE LP type:tp RP {: RESULT = #<type(`tp)>; :}
+ | REPEAT Variable:v EQ expr:s STEP expr:b
+ {: RESULT = #<repeat(lambda(`v,`b),`s)>; :}
+ | REPEAT Variable:v EQ expr:s STEP expr:b LIMIT expr:n
+ {: RESULT = #<repeat(lambda(`v,`b),`s,`n)>; :}
+ | REPEAT LP var_list:vl RP EQ expr:s STEP expr:b LIMIT expr:n
+ {: RESULT = #<loop(lambda(tuple(...vl),`b),`s,`n)>; :}
+ | START_TEMPLATE:s template:t END_TEMPLATE:e
+ {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),...t,text(`e))>); :}
+ | START_TEMPLATE:s END_TEMPLATE:e {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),text(`e))>); :}
+ | const:c {: RESULT = c; :}
+ ;
+template ::= template:t TEXT:s {: RESULT = t.append(#<text(`s)>); :}
+ | template:t expr:e {: RESULT = t.append(e); :}
+ | TEXT:s {: RESULT = #[text(`s)]; :}
+ | expr:e {: RESULT = #[`e]; :}
+ ;
+expl ::= expr:e {: RESULT = #[`e]; :}
+ | expl:el COMMA expr:e {: RESULT = el.append(e); :}
+ ;
+name_binds ::= name_binds:nl
+ COMMA var:v COLON expr:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
+ | var:v COLON expr:e {: RESULT = #[bind(`v,`e)]; :}
+ ;
+cases ::= cases:cs SEP pattern:p COLON expr:e {: RESULT = cs.append(#<case(`p,`e)>); :}
+ | pattern:p COLON expr:e {: RESULT = #[case(`p,`e)]; :}
+ ;
+pattern ::= var:v {: RESULT = v; :}
+ | TIMES {: RESULT = #<any>; :}
+ | const:c {: RESULT = c; :}
+ | LP RP {: RESULT = #<tuple()>; :}
+ | LP pat_list:el RP {: RESULT = (el.length()==1)?el.head():#<tuple(...el)>; :}
+ | LB RB {: RESULT = #<list()>; :}
+ | LB expl:el RB {: RESULT = #<list(...el)>; :}
+ | pattern:p AS type:t {: RESULT = #<typed(`p,`t)>; :}
+ | Variable:c LP pat_list:pl RP {: RESULT = #<call(`c,...pl)>; :}
+ | LT {: MRQLLex.record_begin(); :}
+ pat_binds:nl GTR {: MRQLLex.record_end(); RESULT = #<record(...nl)>; :}
+ | START_TEMPLATE:s template_pat:t END_TEMPLATE:e
+ {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),...t,text(`e))>); :}
+ | START_TEMPLATE:s END_TEMPLATE:e {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),text(`e))>); :}
+ ;
+template_pat ::= template_pat:t TEXT:s {: RESULT = t.append(#<text(`s)>); :}
+ | template_pat:t pattern:e {: RESULT = t.append(e); :}
+ | TEXT:s {: RESULT = #[text(`s)]; :}
+ | pattern:e {: RESULT = #[`e]; :}
+ ;
+pat_list ::= pattern:e {: RESULT = #[`e]; :}
+ | pat_list:el COMMA pattern:e {: RESULT = el.append(e); :}
+ ;
+pat_binds ::= pat_binds:nl
+ COMMA var:v COLON pattern:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
+ | var:v COLON pattern:e {: RESULT = #[bind(`v,`e)]; :}
+ ;
+mrql ::= SELECT opt_distinct:d expr:e
+ FROM binds:bl
+ opt_where:w
+ opt_groupby:g
+ opt_orderby:o {: RESULT = #<select(`d,`e,from(...bl),`w,`g,`o)>; :}
+ | SOME binds:bl COLON expr:e {: RESULT = #<call(exists,select(none,true,from(...bl),where(`e),groupby(),orderby()))>; :}
+ | ALL binds:bl COLON expr:e {: RESULT = #<call(all,select(none,`e,from(...bl),where(true),groupby(),orderby()))>; :}
+ | ALL LP mrql:e RP {: RESULT = #<call(all,`e)>; :}
+ | SOME LP mrql:e RP {: RESULT = #<call(some,`e)>; :}
+ ;
+opt_distinct ::= DISTINCT {: RESULT = #<distinct>; :}
+ | {: RESULT = #<none>; :}
+ ;
+binds ::= binds:bl COMMA pattern:p IN expr:e {: RESULT = bl.append(#<bind(`p,`e)>); :}
+ | binds:bl COMMA pattern:p EQ expr:e {: RESULT = bl.append(#<bind(`p,bag(`e))>); :}
+ | pattern:p IN expr:e {: RESULT = #[bind(`p,`e)]; :}
+ | pattern:p EQ expr:e {: RESULT = #[bind(`p,bag(`e))]; :}
+ ;
+opt_where ::= WHERE expr:e {: RESULT = #<where(`e)>; :}
+ | {: RESULT = #<where(true)>; :}
+ ;
+opt_orderby ::= ORDER BY order_binds:ol opt_limit:l {: RESULT = #<orderby(`l,...ol)>; :}
+ | {: RESULT = #<orderby()>; :}
+ ;
+order_binds ::= expr:e mode:m {: RESULT = m.equals(#<asc>) ? #[`e] : #[call(inv,`e)]; :}
+ | order_binds:ol COMMA expr:e mode:m {: RESULT = ol.append(m.equals(#<asc>) ? #<`e> : #<call(inv,`e)>); :}
+ ;
+mode ::= ASCENDING {: RESULT = #<asc>; :}
+ | DESCENDING {: RESULT = #<desc>; :}
+ | {: RESULT = #<asc>; :}
+ ;
+opt_limit ::= LIMIT expr:n {: RESULT = n; :}
+ | {: RESULT = #<none>; :}
+ ;
+opt_groupby ::= GROUP BY groupby_binds:gl
+ opt_having:h {: RESULT = #<groupby(`h,...gl)>; :}
+ | {: RESULT = #<groupby()>; :}
+ ;
+groupby_binds ::= pattern:p COLON expr:e {: RESULT = #[bind(`p,`e)]; :}
+ | pattern:p {: RESULT = #[bind(`p,`p)]; :}
+ ;
+opt_having ::= HAVING expr:e {: RESULT = e; :}
+ | {: RESULT = #<true>; :}
+ ;
+type ::= var:v {: RESULT = v; :}
+ | LP RP {: RESULT = #<tuple()>; :}
+ | LP typel:el RP {: RESULT = #<tuple(...el)>; :}
+ | LT type_binds:nl GT {: RESULT = #<record(...nl)>; :}
+ | LB type:tp RB {: RESULT = #<list(`tp)>; :}
+ | LSB type:tp RSB {: RESULT = #<bag(`tp)>; :}
+ | Variable:v LP typel:tl RP {: RESULT = #<`v(...tl)>; :}
+ | EXCLAMATION type:tp {: RESULT = #<persistent(`tp)>; :}
+ ;
+typel ::= type:e {: RESULT = #[`e]; :}
+ | typel:el COMMA type:e {: RESULT = el.append(e); :}
+ ;
+type_binds ::= type_binds:nl
+ COMMA var:v COLON type:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
+ | var:v COLON type:e {: RESULT = #[bind(`v,`e)]; :}
+ ;
+data_binds ::= data_binds:nl
+ SEP Variable:c COLON type:e {: RESULT = nl.append(#<`c(`e)>); :}
+ | Variable:c COLON type:e {: RESULT = #[`c(`e)]; :}
+ ;
+xpath ::= DOT {: RESULT = #<dot>; :}
+ | xpath:x DIV xname:n {: RESULT = #<child(`n,`x)>; :}
+ | xpath:x DIV DIV xname:n {: RESULT = #<descendant(`n,`x)>; :}
+ | xpath:x LB Integer:n RB {: RESULT = #<index(`x,`n)>; :}
+ | xpath:x LB xpred:p RB {: RESULT = #<predicate(`p,`x)>; :}
+ | DIV xname:n {: RESULT = #<child(`n,root)>; :}
+ | DIV DIV xname:n {: RESULT = #<descendant(`n,root)>; :}
+ | xname:n {: RESULT = #<child(`n,dot)>; :}
+ ;
+xname ::= var:v {: RESULT = v; :}
+ | Variable:n COLON Variable:v {: RESULT = new VariableLeaf(n+":"+v); :}
+ | TIMES {: RESULT = new VariableLeaf("*"); :}
+ | ATSYM Variable:v {: RESULT = new VariableLeaf("@"+v); :}
+ | ATSYM Variable:n COLON Variable:v {: RESULT = new VariableLeaf("@"+n+":"+v); :}
+ | ATSYM TIMES {: RESULT = new VariableLeaf("@*"); :}
+ | ATSYM Variable:n COLON TIMES {: RESULT = new VariableLeaf("@"+n+":*"); :}
+ ;
+xpred ::= xpath:x EQ const:c {: RESULT = #<eq(`x,`c)>; :}
+ | xpath:x {: RESULT = x; :}
+ ;
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/mrql.lex
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/mrql.lex b/core/src/main/java/org/apache/mrql/mrql.lex
new file mode 100644
index 0000000..01d6509
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/mrql.lex
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java_cup.runtime.Symbol;
+import org.apache.mrql.gen.Tree;
+
+%%
+%class MRQLLex
+%public
+%line
+%char
+%ignorecase
+%cup
+%state COMMENT
+%state TEMPLATE
+%eofval{
+ return symbol(sym.EOF);
+%eofval}
+%{
+
+ static int prev_char_pos = -1;
+
+ public static int[] nest = new int[1000];
+ public static int nest_pos = 0;
+
+ static String template = null;
+
+ public static void reset () {
+ nest_pos = 0;
+ nest[0] = 0;
+ prev_char_pos = -1;
+ }
+
+ public static void record_begin () {
+ nest[++nest_pos] = 0;
+ }
+
+ public static void record_end () {
+ nest_pos--;
+ }
+
+ public int line_pos () { return yyline+1; }
+
+ public int char_pos () { return yychar-prev_char_pos; }
+
+ public Symbol symbol ( int s ) {
+ Tree.line_number = line_pos();
+ Tree.position_number = char_pos();
+ return new Symbol(s);
+ }
+
+ public Symbol symbol ( int s, Object o ) {
+ Tree.line_number = line_pos();
+ Tree.position_number = char_pos();
+ return new Symbol(s,o);
+ }
+
+ public void error ( String msg ) {
+ System.err.println("*** Scanner Error: " + msg + " (line: " + line_pos() + ", position: " + char_pos() + ")");
+ if (Config.testing)
+ throw new Error("Scanner Error");
+ }
+
+ public String format ( String s ) {
+ return s.replaceAll("\\\\t", "\t").replaceAll("\\\\n", "\n");
+ }
+%}
+
+ID=[a-zA-Z][a-zA-Z0-9_]*
+INT = [0-9]+
+DOUBLE = [0-9]+([\.][0-9]+)?([eE][+-]?[0-9]+)?
+
+%%
+
+<YYINITIAL> {INT} { return symbol(sym.Integer,new Long(yytext())); }
+<YYINITIAL> {DOUBLE} { return symbol(sym.Double,new Double(yytext())); }
+<YYINITIAL> "["{ID}"|" { yybegin(TEMPLATE); template = ""; return symbol(sym.START_TEMPLATE,yytext()); }
+<YYINITIAL> "if" { return symbol(sym.IF); }
+<YYINITIAL> "then" { return symbol(sym.THEN); }
+<YYINITIAL> "else" { return symbol(sym.ELSE); }
+<YYINITIAL> "select" { return symbol(sym.SELECT); }
+<YYINITIAL> "from" { return symbol(sym.FROM); }
+<YYINITIAL> "having" { return symbol(sym.HAVING); }
+<YYINITIAL> "[" { nest[nest_pos]++; return symbol(sym.LB); }
+<YYINITIAL> "]" { nest[nest_pos]--; return symbol(sym.RB); }
+<YYINITIAL> "(" { nest[nest_pos]++; return symbol(sym.LP); }
+<YYINITIAL> ")" { nest[nest_pos]--; return symbol(sym.RP); }
+<YYINITIAL> "{" { nest[nest_pos]++; return symbol(sym.LSB); }
+<YYINITIAL> "}}" { nest[nest_pos]--; if (nest_pos > 0 && nest[nest_pos] == 0) { record_end(); yybegin(TEMPLATE); } else return symbol(sym.RSB); }
+<YYINITIAL> "}" { nest[nest_pos]--; return symbol(sym.RSB); }
+<YYINITIAL> "+" { return symbol(sym.PLUS); }
+<YYINITIAL> "-" { return symbol(sym.MINUS); }
+<YYINITIAL> "*" { return symbol(sym.TIMES); }
+<YYINITIAL> "/" { return symbol(sym.DIV); }
+<YYINITIAL> "mod" { return symbol(sym.MOD); }
+<YYINITIAL> "%" { return symbol(sym.MOD); }
+<YYINITIAL> "=" { return symbol(sym.EQ); }
+<YYINITIAL> "<>" { return symbol(sym.NEQ); }
+<YYINITIAL> "<=" { return symbol(sym.LEQ); }
+<YYINITIAL> "<" { return symbol(sym.LT); }
+<YYINITIAL> ">=" { return symbol(sym.GEQ); }
+<YYINITIAL> "!" { return symbol(sym.EXCLAMATION); }
+<YYINITIAL> ">" { return (nest_pos > 0 && nest[nest_pos] == 0) ? symbol(sym.GTR) : symbol(sym.GT); }
+<YYINITIAL> \\ { return symbol(sym.BSLASH); }
+<YYINITIAL> "and" { return symbol(sym.AND); }
+<YYINITIAL> "or" { return symbol(sym.OR); }
+<YYINITIAL> "not" { return symbol(sym.NOT); }
+<YYINITIAL> "union" { return symbol(sym.UNION); }
+<YYINITIAL> "intersect" { return symbol(sym.INTERSECT); }
+<YYINITIAL> "except" { return symbol(sym.EXCEPT); }
+<YYINITIAL> "exists" { return symbol(sym.EXISTS); }
+<YYINITIAL> "in" { return symbol(sym.IN); }
+<YYINITIAL> "let" { return symbol(sym.LET); }
+<YYINITIAL> "," { return symbol(sym.COMMA); }
+<YYINITIAL> "." { return symbol(sym.DOT); }
+<YYINITIAL> ":=" { return symbol(sym.ASSIGN); }
+<YYINITIAL> ":" { return symbol(sym.COLON); }
+<YYINITIAL> ";" { return symbol(sym.SEMI); }
+<YYINITIAL> "#" { return symbol(sym.SHARP); }
+<YYINITIAL> "@" { return symbol(sym.ATSYM); }
+<YYINITIAL> \| { return symbol(sym.SEP); }
+<YYINITIAL> "where" { return symbol(sym.WHERE); }
+<YYINITIAL> "order" { return symbol(sym.ORDER); }
+<YYINITIAL> "group" { return symbol(sym.GROUP); }
+<YYINITIAL> "by" { return symbol(sym.BY); }
+<YYINITIAL> "asc" { return symbol(sym.ASCENDING); }
+<YYINITIAL> "desc" { return symbol(sym.DESCENDING); }
+<YYINITIAL> "function" { return symbol(sym.FUNCTION); }
+<YYINITIAL> "macro" { return symbol(sym.MACRO); }
+<YYINITIAL> "distinct" { return symbol(sym.DISTINCT); }
+<YYINITIAL> "as" { return symbol(sym.AS); }
+<YYINITIAL> "some" { return symbol(sym.SOME); }
+<YYINITIAL> "all" { return symbol(sym.ALL); }
+<YYINITIAL> "store" { return symbol(sym.STORE); }
+<YYINITIAL> "dump" { return symbol(sym.DUMP); }
+<YYINITIAL> "type" { return symbol(sym.TYPE); }
+<YYINITIAL> "data" { return symbol(sym.DATA); }
+<YYINITIAL> "case" { return symbol(sym.CASE); }
+<YYINITIAL> "xpath" { return symbol(sym.XPATH); }
+<YYINITIAL> "repeat" { return symbol(sym.REPEAT); }
+<YYINITIAL> "step" { return symbol(sym.STEP); }
+<YYINITIAL> "limit" { return symbol(sym.LIMIT); }
+<YYINITIAL> "import" { return symbol(sym.IMPORT); }
+<YYINITIAL> "parser" { return symbol(sym.PARSER); }
+<YYINITIAL> "include" { return symbol(sym.INCLUDE); }
+<YYINITIAL> "aggregation" { return symbol(sym.AGGREGATION); }
+
+<YYINITIAL> {ID} { return symbol(sym.Variable,yytext()); }
+
+<YYINITIAL> "/*" { yybegin(COMMENT); }
+<COMMENT> "*/" { yybegin(YYINITIAL); }
+<COMMENT> [ \t\f] { }
+<COMMENT> [\r\n] { prev_char_pos = yychar; }
+<COMMENT> . { }
+
+<TEMPLATE> "|]" { yybegin(YYINITIAL); String s = template; template = ""; return symbol(sym.END_TEMPLATE,s); }
+<TEMPLATE> "{{" { yybegin(YYINITIAL); record_begin(); nest[nest_pos]++; String s = template; template = ""; return symbol(sym.TEXT,s); }
+<TEMPLATE> [ \t\f] { template += yytext(); }
+<TEMPLATE> [\r\n] { template += yytext(); prev_char_pos = yychar; }
+<TEMPLATE> . { template += yytext(); }
+
+<YYINITIAL> "//"[^\n]*\n { prev_char_pos = yychar; }
+
+<YYINITIAL> \"[^\"]*\" { return symbol(sym.String,format(yytext().substring(1,yytext().length()-1))); }
+<YYINITIAL> \'[^\']*\' { return symbol(sym.String,format(yytext().substring(1,yytext().length()-1))); }
+
+<YYINITIAL> [ \t\f] { }
+<YYINITIAL> [\r\n] { prev_char_pos = yychar; }
+
+<YYINITIAL> . { error("Illegal character: "+yytext()); }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/pom.xml
----------------------------------------------------------------------
diff --git a/gen/pom.xml b/gen/pom.xml
new file mode 100644
index 0000000..d49ad1c
--- /dev/null
+++ b/gen/pom.xml
@@ -0,0 +1,87 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-gen</artifactId>
+ <packaging>jar</packaging>
+ <name>Apache MRQL gen</name>
+ <description>Apache MRQL Gen: A Java Package for Constructing and Manipulating Abstract Syntax Trees</description>
+ <url>http://mrql.incubator.apache.org/</url>
+ <inceptionYear>2013</inceptionYear>
+
+ <parent>
+ <groupId>org.apache.mrql</groupId>
+ <artifactId>mrql-parent</artifactId>
+ <version>0.9.1-incubating-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.4</version>
+ <configuration>
+ <finalName>mrql-gen-${project.version}</finalName>
+ <outputDirectory>${project.parent.basedir}/lib</outputDirectory>
+ <useDefaultManifestFile>true</useDefaultManifestFile>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>de.jflex</groupId>
+ <artifactId>maven-jflex-plugin</artifactId>
+ <version>1.4.3</version>
+ <executions>
+ <execution>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <lexDefinitions>
+ <lexDefinition>src/main/java/org/apache/mrql/gen/gen.lex</lexDefinition>
+ </lexDefinitions>
+ <outputDirectory>${project.build.directory}/generated-sources/parsers</outputDirectory>
+ <verbose>false</verbose>
+ <backup>false</backup>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <version>1.7</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <target>
+ <taskdef name="cup" classname="java_cup.anttask.CUPTask" classpathref="maven.compile.classpath" />
+ <cup srcfile="${project.basedir}/src/main/java/org/apache/mrql/gen/gen.cup" destdir="${project.build.directory}/generated-sources/parsers" nosummary="true" parser="GenParser" symbols="GenSym" />
+ </target>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Condition.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Condition.java b/gen/src/main/java/org/apache/mrql/gen/Condition.java
new file mode 100644
index 0000000..6b28585
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Condition.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+
+
+final public class Condition {
+ public final String stmt;
+ public final String pred;
+ public final int unmatched_brackets;
+ public Condition ( String s, String p, int n ) {
+ stmt = s;
+ pred = p;
+ unmatched_brackets = n;
+ }
+ public String and () {
+ if (pred.equals("true"))
+ return "";
+ else return " && " + pred;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Crypt.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Crypt.java b/gen/src/main/java/org/apache/mrql/gen/Crypt.java
new file mode 100644
index 0000000..b6f95a7
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Crypt.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+abstract public class Crypt {
+ public static boolean encryptp = false;
+ public static String crypt ( String s ) {
+ byte[] b = s.getBytes();
+ for (int i=0; i<b.length; i++)
+ { if (b[i]<45)
+ b[i] = (byte) (b[i]+78);
+ else b[i] = (byte) (b[i]-13);
+ if (b[i]==34)
+ b[i] = 123;
+ else if (b[i]==92)
+ b[i] = 124;
+ else if (b[i]=='\n')
+ b[i] = 125;
+ };
+ return new String(b);
+ }
+ public static String decrypt ( String s ) {
+ byte[] b = s.getBytes();
+ for (int i=0; i<b.length; i++)
+ { if (b[i]==123)
+ b[i] = 47;
+ else if (b[i]==124)
+ b[i] = 105;
+ else if (b[i]==125)
+ b[i] = '\n';
+ else if (b[i]>109)
+ b[i] = (byte) (b[i]-78);
+ else b[i] = (byte) (b[i]+13);
+ };
+ return new String(b);
+ }
+ public static String quotes ( String s ) { return "\"" + s + "\""; }
+ public static String encrypt ( String s ) {
+ if (encryptp)
+ return "Crypt.decrypt(\"" + crypt(s) + "\")";
+ else return quotes(s);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/DoubleLeaf.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/DoubleLeaf.java b/gen/src/main/java/org/apache/mrql/gen/DoubleLeaf.java
new file mode 100644
index 0000000..ba9ac90
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/DoubleLeaf.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+final public class DoubleLeaf extends Tree {
+ public double value;
+ public DoubleLeaf ( float n ) {
+ super();
+ value = (double)n;
+ }
+ public DoubleLeaf ( double n ) {
+ super();
+ value = n;
+ }
+ public double value () { return value; }
+ public boolean equals ( Tree e ) {
+ return (e instanceof DoubleLeaf)
+ && value==((DoubleLeaf) e).value;
+ }
+ protected int size () { return toString().length(); }
+ public String toString () {
+ return Double.toString(value);
+ }
+ public String pretty ( int position ) {
+ return toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/LongLeaf.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/LongLeaf.java b/gen/src/main/java/org/apache/mrql/gen/LongLeaf.java
new file mode 100644
index 0000000..6611f32
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/LongLeaf.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+
+
+final public class LongLeaf extends Tree {
+ public long value;
+
+ public LongLeaf ( int n ) {
+ super();
+ value = (long)n;
+ }
+
+ public LongLeaf ( long n ) {
+ super();
+ value = n;
+ }
+
+ public long value () { return value; }
+
+ public boolean equals ( Tree e ) {
+ return (e instanceof LongLeaf)
+ && value == ((LongLeaf) e).value;
+ }
+
+ protected int size () { return toString().length(); }
+
+ public String toString () {
+ return Long.toString(value);
+ }
+
+ public String pretty ( int position ) {
+ return toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Main.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Main.java b/gen/src/main/java/org/apache/mrql/gen/Main.java
new file mode 100644
index 0000000..1d4f91e
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Main.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java_cup.runtime.*;
+import java.io.*;
+import java.util.ArrayList;
+
+public class Main extends GenParser {
+ public static void main ( String args[] ) throws Exception {
+ ArrayList<String> files = new ArrayList<String>();
+ boolean is_directory = false;
+ String output = null;
+ for ( int i = 0; i < args.length; i++ )
+ if (args[i].equals("-encrypt"))
+ Crypt.encryptp = true;
+ else if (args[i].equals("-o"))
+ output = args[++i];
+ else files.add(args[i]);
+ if (output != null && new File(output).isDirectory())
+ is_directory = true;
+ if (!is_directory && files.size() > 1)
+ throw new Error("Expected an output directory: "+output);
+ for ( String file: files )
+ try {
+ Meta.clear();
+ scanner = new GenLex(new FileInputStream(file));
+ String outfile = file.replace(".gen",".java");
+ if (is_directory)
+ outfile = new File(output,new File(outfile).getName()).getPath();
+ if (outfile.equals(file))
+ if (is_directory) {
+ System.err.println("Cannot compile: "+file);
+ continue;
+ } else outfile = output;
+ out = new PrintStream(new FileOutputStream(outfile));
+ out.print("/* DO NOT EDIT THIS FILE. THIS FILE WAS GENERATED FROM "+file+" BY GEN */");
+ new GenParser(scanner).parse();
+ Meta.dump_names(out);
+ out.close();
+ } catch (Error ex) {
+ System.err.println(ex.getMessage()+" while parsing the GEN file: "+file);
+ System.exit(-1);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/gen/src/main/java/org/apache/mrql/gen/Meta.java
----------------------------------------------------------------------
diff --git a/gen/src/main/java/org/apache/mrql/gen/Meta.java b/gen/src/main/java/org/apache/mrql/gen/Meta.java
new file mode 100644
index 0000000..80d31bd
--- /dev/null
+++ b/gen/src/main/java/org/apache/mrql/gen/Meta.java
@@ -0,0 +1,260 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql.gen;
+
+import java.io.*;
+import java.util.HashMap;
+import java.util.Random;
+
+
+final public class Meta extends Crypt {
+
+ final public static Tree mark = new LongLeaf(-1);
+
+ static int level = 0;
+
+ private static int name_counter = 0;
+
+ public static Tree new_name () {
+ return new VariableLeaf("N" + (name_counter++) + "_");
+ }
+
+ public static Tree escape ( Tree e ) {
+ if (Tree.parsed) {
+ Tree.line_number = e.line;
+ Tree.position_number = e.position;
+ };
+ return e;
+ }
+
+ public static void clear () {
+ name_counter = 0;
+ name_index = 0;
+ name_vars.clear();
+ package_name = "PatternNames_"+Math.abs(random.nextLong());
+ }
+
+ public static Tree escape ( String s ) { return new VariableLeaf(s); }
+
+ public static Tree escape ( long n ) { return new LongLeaf(n); }
+
+ public static Tree escape ( double n ) { return new DoubleLeaf(n); }
+
+ private static HashMap<String,Integer> name_vars = new HashMap<String,Integer>(1000);
+
+ private static int name_index = 0;
+
+ private final static Random random = new Random();
+
+ private static String package_name = "PatternNames_"+Math.abs(random.nextLong());
+
+ private static String name ( String s ) {
+ Integer ns = name_vars.get(s);
+ if (ns == null) {
+ ns = new Integer(name_index++);
+ name_vars.put(s,ns);
+ };
+ return package_name+".P_"+ns.intValue();
+ }
+
+ static void dump_names ( PrintStream out ) {
+ if (name_vars.isEmpty())
+ return;
+ out.println("abstract class "+package_name+" {");
+ for ( String s: name_vars.keySet() )
+ out.println(" final static String P_"+name_vars.get(s)+" = Tree.add("+s+");");
+ out.println("}");
+ }
+
+ public static Trees subst_list ( Tree term, Tree value, Trees es ) {
+ Trees res = Trees.nil;
+ for (Trees r = es; !r.is_empty(); r = r.tail)
+ res = res.cons(subst_expr(term,value,r.head));
+ return res.reverse();
+ }
+
+ public static Tree subst_expr ( Tree term, Tree value, Tree e ) {
+ if (e.equals(term))
+ return value;
+ else if (e instanceof Node)
+ return new Node(((Node)e).name,
+ subst_list(term,value,((Node)e).children));
+ else return e;
+ }
+
+ public static Tree substitute ( Trees path, Tree e, Tree value ) {
+ if (path.is_empty())
+ return value;
+ else {
+ Trees res = Trees.nil;
+ for (Trees r = ((Node)e).children; !r.is_empty(); r = r.tail)
+ if (r.head.equals(path.head))
+ res = res.cons(substitute(path.tail,r.head,value));
+ else res = res.cons(r.head);
+ return new Node(((Node)e).name,res.reverse());
+ }
+ }
+
+ public static String reify ( Tree e ) {
+ if (e instanceof LongLeaf)
+ return "new LongLeaf(" + e + ")";
+ else if (e instanceof DoubleLeaf)
+ return "new DoubleLeaf(" + e + ")";
+ else if (e instanceof VariableLeaf)
+ if (((VariableLeaf)e).value.equals("_any_"))
+ throw new Error("Gen: Cannot use \"_\" (any) in Tree Construction: "+e);
+ else return "new VariableLeaf(" + encrypt(((VariableLeaf)e).value) + ")";
+ else if (e instanceof StringLeaf)
+ return "new StringLeaf(" + e + ")";
+ else {
+ Node n = (Node) e;
+ if (n.name.equals("Node")) {
+ String s = "new Node(";
+ if (n.children.head instanceof VariableLeaf)
+ if (((VariableLeaf)n.children.head).value.equals("_any_"))
+ throw new Error("Gen: Cannot use \"_\" (any) in AST node name: "+e);
+ else s = s + encrypt(((VariableLeaf)n.children.head).value);
+ else {
+ Node m = (Node) n.children.head;
+ if (m.name.equals("Code"))
+ s = s + ((StringLeaf)m.children.head).value;
+ else if (m.name.equals("Escape"))
+ s = s + ((VariableLeaf)m.children.head).value;
+ };
+ s = s + ",Trees.nil";
+ for (Trees r= n.children.tail; !r.is_empty(); r = r.tail)
+ if (r.head instanceof Node && ((Node)r.head).name.equals("Dots")) {
+ Node m = (Node) r.head;
+ if (m.children.is_empty())
+ throw new Error("Gen: Cannot use \"...\" in Tree construction: "+e);
+ else if (m.children.head instanceof VariableLeaf)
+ s = s + ".append(" + ((VariableLeaf)m.children.head).value + ")";
+ else s = s + ".append(" + ((StringLeaf)m.children.head).value + ")";
+ } else s = s + ".append(" + reify(r.head) + ")";
+ return s + ")";
+ } else if (n.name.equals("Code"))
+ return "Meta.escape(" + ((StringLeaf)n.children.head).value + ")";
+ else if (n.name.equals("Escape"))
+ return "Meta.escape(" + ((VariableLeaf)n.children.head).value + ")";
+ else if (n.name.equals("Higher"))
+ return "Meta.substitute(" + ((VariableLeaf)n.children.head).value
+ + ".tail," + ((VariableLeaf)n.children.head).value
+ + ".head," + reify(n.children.tail.head) + ")";
+ else throw new Error("Gen: Wrong Tree construction: "+e);
+ }
+ }
+
+ public static Condition pattern_list ( Trees args, String v ) {
+ if (args.is_empty())
+ return new Condition("",v+".tail==null",0);
+ else if (args.head instanceof Node && ((Node)args.head).name.equals("Dots")) {
+ if (args.tail.is_empty()) {
+ Node m = (Node) args.head;
+ if (m.children.is_empty())
+ return new Condition("","true",0);
+ else if (m.children.head instanceof VariableLeaf)
+ return new Condition("Trees " + ((VariableLeaf)m.children.head).value + " = " + v + "; ",
+ "true",0);
+ else if (m.children.head instanceof StringLeaf)
+ new Condition("",v + ".head.equals(" + ((StringLeaf)m.children.head).value + ")",0);
+ return new Condition("","true",0);
+ } else {
+ Node m = (Node) args.head;
+ if (m.children.is_empty()) {
+ Condition rest = pattern_list(args.tail,"R_");
+ return new Condition("Trees R_ = " + v + "; for(; R_.tail!=null && !(" + rest.pred
+ + "); R_=R_.tail) ; if (R_.tail!=null) { " + rest.stmt,
+ "true",rest.unmatched_brackets+1);
+ } else if (m.children.head instanceof VariableLeaf) {
+ String nm = ((VariableLeaf)m.children.head).value;
+ Condition rest = pattern_list(args.tail,nm+"_");
+ return new Condition("Trees " + nm + " = Trees.nil; Trees " + nm + "_=" + v + "; "
+ + "for(; " + nm + "_.tail!=null && !(" + rest.pred + "); "
+ + nm + "_=(FOUND_" + Meta.level + ")?" + nm + "_:" + nm + "_.tail) " + nm + " = " + nm + ".append("
+ + nm + "_.head); " + "if (" + nm + "_.tail!=null) { " + rest.stmt,
+ "true",rest.unmatched_brackets+1);
+ } else if (m.children.head instanceof StringLeaf) {
+ Condition rest = pattern_list(args.tail,v);
+ return new Condition(rest.stmt,
+ v + ".equals(" + ((StringLeaf)m.children.head).value + ")" + rest.and(),
+ rest.unmatched_brackets);
+ };
+ return new Condition("","true",0);
+ }
+ } else {
+ Condition c = pattern(args.head,v+".head");
+ Condition rest = pattern_list(args.tail,v+".tail");
+ return new Condition(c.stmt + rest.stmt,
+ v + ".tail!=null" + c.and() + rest.and(),
+ c.unmatched_brackets+rest.unmatched_brackets);
+ }
+ }
+
+ public static Condition pattern ( Tree e, String v ) {
+ if (e instanceof LongLeaf)
+ return new Condition("", "(" + v + " instanceof LongLeaf) && ((LongLeaf)" + v + ").value==" + e,0);
+ else if (e instanceof DoubleLeaf)
+ return new Condition("","(" + v + " instanceof DoubleLeaf) && ((DoubleLeaf)" + v + ").value==" + e,0);
+ else if (e instanceof VariableLeaf)
+ if (((VariableLeaf)e).value.equals("_any_"))
+ return new Condition("","true",0);
+ else return new Condition("","(" + v + " instanceof VariableLeaf) && ((VariableLeaf)" + v
+ + ").value==" + name(encrypt(((VariableLeaf)e).value)),0);
+ else if (e instanceof StringLeaf)
+ return new Condition("","(" + v + " instanceof StringLeaf) && ((StringLeaf)" + v + ").value.equals(" + e + ")",0);
+ else {
+ Node n = (Node) e;
+ if (n.name.equals("Node")) {
+ String p = "(" + v + " instanceof Node)";
+ String s = "";
+ if (n.children.head instanceof VariableLeaf) {
+ if (((VariableLeaf)n.children.head).value!="_any_")
+ p = p + " && ((Node)" + v + ").name=="
+ + name(encrypt(((VariableLeaf)n.children.head).value));
+ } else if (n.children.head instanceof Node)
+ if (((Node)n.children.head).name.equals("Escape"))
+ s = "String " + ((VariableLeaf)((Node)n.children.head).children.head).value
+ + " = ((Node)" + v + ").name; ";
+ else throw new Error("Gen: Wrong Tree pattern: "+e);
+ Condition c = pattern_list(n.children.tail,"((Node)" + v + ").children");
+ return new Condition(s+c.stmt,p+c.and(),c.unmatched_brackets);
+ } else if (n.name.equals("Escape"))
+ return new Condition("Tree " + ((VariableLeaf)n.children.head).value + " = " + v + "; ",
+ "true",0);
+ else if (n.name.equals("IS")) {
+ String nm = ((VariableLeaf)n.children.head).value;
+ Condition c = pattern(n.children.tail.head,nm);
+ return new Condition(c.stmt + "Tree " + nm + " = " + v + "; ",c.pred,0);
+ } else if (n.name.equals("Code"))
+ return new Condition("",v + ".equals(" + ((StringLeaf)n.children.head).value + ")",0);
+ else if (n.name.equals("Higher")) {
+ String nm = ((VariableLeaf)n.children.head).value;
+ Condition c = pattern(n.children.tail.head,nm+"__");
+ String s = reify(n.children.tail.head);
+ return new Condition("Tree " + nm + "_ = " + v + "; Trees " + nm + " = Trees.nil; Trees STACK_" + nm
+ + "_ = new Trees(" + nm + "_); while (!FOUND_" + Meta.level + " && STACK_" + nm + "_.tail!=null) { Tree "
+ + nm + "__ = STACK_" + nm + "_.head; STACK_" + nm + "_ = STACK_" + nm + "_.tail; " + nm
+ + " = (" + nm + "__==Meta.mark) ? " + nm + ".tail : ((" + nm + "__ instanceof Node) ? "
+ + nm + ".cons(" + nm + "__) : " + nm + "); if (" + nm + "__ instanceof Node) STACK_" + nm
+ + "_ = ((Node)" + nm + "__).children.append(STACK_" + nm + "_.cons(Meta.mark)); if ("
+ + nm + "__!=Meta.mark" + c.and() + ") { if (!(" + nm + "__ instanceof Node)) " + nm + "="
+ + nm + ".cons(" + nm + "__); " + nm + " = " + nm + ".reverse(); " + c.stmt,"true",2);
+ } else throw new Error("Gen: Wrong Tree pattern: "+e);
+ }
+ }
+}
[19/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/SystemFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/SystemFunctions.java b/core/src/main/java/org/apache/mrql/SystemFunctions.java
new file mode 100644
index 0000000..5c0ed4b
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/SystemFunctions.java
@@ -0,0 +1,467 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.util.*;
+import java.lang.Math;
+
+
+/** System functions must be from MRData to MRData */
+final public class SystemFunctions {
+
+ // used for shortcutting sync in bsp supersteps
+ final public static Bag bsp_empty_bag = new Bag();
+ final public static MR_bool bsp_true_value = new MR_bool(true);
+ final public static MR_bool bsp_false_value = new MR_bool(false);
+
+ final static MRData null_value = new Tuple(0);
+ final static MR_bool true_value = new MR_bool(true);
+ final static MR_bool false_value = new MR_bool(false);
+
+ private static void error ( String s ) {
+ System.err.println("*** "+s);
+ throw new Error(s);
+ }
+
+ static Random random = new Random();
+
+ public static MR_bool synchronize ( MR_string peerName, MR_bool mr_exit ) {
+ return Evaluator.evaluator.synchronize(peerName,mr_exit);
+ }
+
+ public static Bag distribute ( MR_string peerName, Bag s ) {
+ return Evaluator.evaluator.distribute(peerName,s);
+ }
+
+ public static MR_bool lt ( MR_short x, MR_short y ) { return (x.get() < y.get()) ? true_value : false_value; }
+ public static MR_bool lt ( MR_int x, MR_int y ) { return (x.get() < y.get()) ? true_value : false_value; }
+ public static MR_bool lt ( MR_long x, MR_long y ) { return (x.get() < y.get()) ? true_value : false_value; }
+ public static MR_bool lt ( MR_float x, MR_float y ) { return (x.get() < y.get()) ? true_value : false_value; }
+ public static MR_bool lt ( MR_double x, MR_double y ) { return (x.get() < y.get()) ? true_value : false_value; }
+
+ public static MR_bool gt ( MR_short x, MR_short y ) { return (x.get() > y.get()) ? true_value : false_value; }
+ public static MR_bool gt ( MR_int x, MR_int y ) { return (x.get() > y.get()) ? true_value : false_value; }
+ public static MR_bool gt ( MR_long x, MR_long y ) { return (x.get() > y.get()) ? true_value : false_value; }
+ public static MR_bool gt ( MR_float x, MR_float y ) { return (x.get() > y.get()) ? true_value : false_value; }
+ public static MR_bool gt ( MR_double x, MR_double y ) { return (x.get() > y.get()) ? true_value : false_value; }
+
+ public static MR_bool leq ( MR_short x, MR_short y ) { return (x.get() <= y.get()) ? true_value : false_value; }
+ public static MR_bool leq ( MR_int x, MR_int y ) { return (x.get() <= y.get()) ? true_value : false_value; }
+ public static MR_bool leq ( MR_long x, MR_long y ) { return (x.get() <= y.get()) ? true_value : false_value; }
+ public static MR_bool leq ( MR_float x, MR_float y ) { return (x.get() <= y.get()) ? true_value : false_value; }
+ public static MR_bool leq ( MR_double x, MR_double y ) { return (x.get() <= y.get()) ? true_value : false_value; }
+
+ public static MR_bool geq ( MR_short x, MR_short y ) { return (x.get() >= y.get()) ? true_value : false_value; }
+ public static MR_bool geq ( MR_int x, MR_int y ) { return (x.get() >= y.get()) ? true_value : false_value; }
+ public static MR_bool geq ( MR_long x, MR_long y ) { return (x.get() >= y.get()) ? true_value : false_value; }
+ public static MR_bool geq ( MR_float x, MR_float y ) { return (x.get() >= y.get()) ? true_value : false_value; }
+ public static MR_bool geq ( MR_double x, MR_double y ) { return (x.get() >= y.get()) ? true_value : false_value; }
+
+ public static MR_bool eq ( MR_short x, MR_short y ) { return (x.get() == y.get()) ? true_value : false_value; }
+ public static MR_bool eq ( MR_int x, MR_int y ) { return (x.get() == y.get()) ? true_value : false_value; }
+ public static MR_bool eq ( MR_long x, MR_long y ) { return (x.get() == y.get()) ? true_value : false_value; }
+ public static MR_bool eq ( MR_float x, MR_float y ) { return (x.get() == y.get()) ? true_value : false_value; }
+ public static MR_bool eq ( MR_double x, MR_double y ) { return (x.get() == y.get()) ? true_value : false_value; }
+
+ public static MR_bool neq ( MR_short x, MR_short y ) { return (x.get() != y.get()) ? true_value : false_value; }
+ public static MR_bool neq ( MR_int x, MR_int y ) { return (x.get() != y.get()) ? true_value : false_value; }
+ public static MR_bool neq ( MR_long x, MR_long y ) { return (x.get() != y.get()) ? true_value : false_value; }
+ public static MR_bool neq ( MR_float x, MR_float y ) { return (x.get() != y.get()) ? true_value : false_value; }
+ public static MR_bool neq ( MR_double x, MR_double y ) { return (x.get() != y.get()) ? true_value : false_value; }
+
+ public static MR_bool eq ( MRData x, MRData y ) { return (x.compareTo(y) == 0) ? true_value : false_value; }
+ public static MR_bool neq ( MRData x, MRData y ) { return (x.compareTo(y) != 0) ? true_value : false_value; }
+ public static MR_bool lt ( MRData x, MRData y ) { return (x.compareTo(y) < 0) ? true_value : false_value; }
+ public static MR_bool leq ( MRData x, MRData y ) { return (x.compareTo(y) <= 0) ? true_value : false_value; }
+ public static MR_bool gt ( MRData x, MRData y ) { return (x.compareTo(y) > 0) ? true_value : false_value; }
+ public static MR_bool geq ( MRData x, MRData y ) { return (x.compareTo(y) >= 0) ? true_value : false_value; }
+
+ public static MR_string plus ( MR_string x, MR_string y ) { return new MR_string(x.get()+y.get()); }
+
+ public static Bag plus ( Bag x, Bag y ) { return x.union(y); }
+
+ public static MR_short plus ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()+y.get())); }
+ public static MR_int plus ( MR_int x, MR_int y ) { return new MR_int(x.get()+y.get()); }
+ public static MR_long plus ( MR_long x, MR_long y ) { return new MR_long(x.get()+y.get()); }
+ public static MR_float plus ( MR_float x, MR_float y ) { return new MR_float(x.get()+y.get()); }
+ public static MR_double plus ( MR_double x, MR_double y ) { return new MR_double(x.get()+y.get()); }
+
+ public static MR_short minus ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()-y.get())); }
+ public static MR_int minus ( MR_int x, MR_int y ) { return new MR_int(x.get()-y.get()); }
+ public static MR_long minus ( MR_long x, MR_long y ) { return new MR_long(x.get()-y.get()); }
+ public static MR_float minus ( MR_float x, MR_float y ) { return new MR_float(x.get()-y.get()); }
+ public static MR_double minus ( MR_double x, MR_double y ) { return new MR_double(x.get()-y.get()); }
+
+ public static MR_short times ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()*y.get())); }
+ public static MR_int times ( MR_int x, MR_int y ) { return new MR_int(x.get()*y.get()); }
+ public static MR_long times ( MR_long x, MR_long y ) { return new MR_long(x.get()*y.get()); }
+ public static MR_float times ( MR_float x, MR_float y ) { return new MR_float(x.get()*y.get()); }
+ public static MR_double times ( MR_double x, MR_double y ) { return new MR_double(x.get()*y.get()); }
+
+ public static MR_short div ( MR_short x, MR_short y ) { return new MR_short((short)(x.get()/y.get())); }
+ public static MR_int div ( MR_int x, MR_int y ) { return new MR_int(x.get()/y.get()); }
+ public static MR_long div ( MR_long x, MR_long y ) { return new MR_long(x.get()/y.get()); }
+ public static MR_float div ( MR_float x, MR_float y ) { return new MR_float(x.get()/y.get()); }
+ public static MR_double div ( MR_double x, MR_double y ) { return new MR_double(x.get()/y.get()); }
+
+ public static MR_int mod ( MR_int x, MR_int y ) { return new MR_int(x.get() % y.get()); }
+ public static MR_long mod ( MR_long x, MR_long y ) { return new MR_long(x.get() % y.get()); }
+
+ public static MR_short min ( MR_short x, MR_short y ) { return new MR_short((short)(Math.min(x.get(),y.get()))); }
+ public static MR_int min ( MR_int x, MR_int y ) { return new MR_int(Math.min(x.get(),y.get())); }
+ public static MR_long min ( MR_long x, MR_long y ) { return new MR_long(Math.min(x.get(),y.get())); }
+ public static MR_float min ( MR_float x, MR_float y ) { return new MR_float(Math.min(x.get(),y.get())); }
+ public static MR_double min ( MR_double x, MR_double y ) { return new MR_double(Math.min(x.get(),y.get())); }
+
+ public static MR_short max ( MR_short x, MR_short y ) { return new MR_short((short)(Math.max(x.get(),y.get()))); }
+ public static MR_int max ( MR_int x, MR_int y ) { return new MR_int(Math.max(x.get(),y.get())); }
+ public static MR_long max ( MR_long x, MR_long y ) { return new MR_long(Math.max(x.get(),y.get())); }
+ public static MR_float max ( MR_float x, MR_float y ) { return new MR_float(Math.max(x.get(),y.get())); }
+ public static MR_double max ( MR_double x, MR_double y ) { return new MR_double(Math.max(x.get(),y.get())); }
+
+ public static MR_double sin ( MR_double x ) { return new MR_double(Math.sin(x.get())); }
+ public static MR_double cos ( MR_double x ) { return new MR_double(Math.cos(x.get())); }
+ public static MR_double tan ( MR_double x ) { return new MR_double(Math.tan(x.get())); }
+ public static MR_double asin ( MR_double x ) { return new MR_double(Math.asin(x.get())); }
+ public static MR_double acos ( MR_double x ) { return new MR_double(Math.acos(x.get())); }
+ public static MR_double atan ( MR_double x ) { return new MR_double(Math.atan(x.get())); }
+ public static MR_double pow ( MR_double x, MR_double y ) { return new MR_double(Math.pow(x.get(),y.get())); }
+ public static MR_double sqrt ( MR_double x ) { return new MR_double(Math.sqrt(x.get())); }
+ public static MR_double ceil ( MR_double x ) { return new MR_double(Math.ceil(x.get())); }
+ public static MR_double floor ( MR_double x ) { return new MR_double(Math.floor(x.get())); }
+ public static MR_double rint ( MR_double x ) { return new MR_double(Math.rint(x.get())); }
+ public static MR_int round ( MR_float x ) { return new MR_int((int)Math.round(x.get())); }
+ public static MR_long round ( MR_double x ) { return new MR_long(Math.round(x.get())); }
+
+ public static MR_short plus ( MR_short x ) { return x; }
+ public static MR_int plus ( MR_int x ) { return x; }
+ public static MR_long plus ( MR_long x ) { return x; }
+ public static MR_float plus ( MR_float x ) { return x; }
+ public static MR_double plus ( MR_double x ) { return x; }
+
+ public static MR_short minus ( MR_short x ) { return new MR_short((short)-x.get()); }
+ public static MR_int minus ( MR_int x ) { return new MR_int(-x.get()); }
+ public static MR_long minus ( MR_long x ) { return new MR_long(-x.get()); }
+ public static MR_float minus ( MR_float x ) { return new MR_float(-x.get()); }
+ public static MR_double minus ( MR_double x ) { return new MR_double(-x.get()); }
+
+ public static MR_short abs ( MR_short x ) { return new MR_short((short) Math.abs(x.get())); }
+ public static MR_int abs ( MR_int x ) { return new MR_int(Math.abs(x.get())); }
+ public static MR_long abs ( MR_long x ) { return new MR_long(Math.abs(x.get())); }
+ public static MR_float abs ( MR_float x ) { return new MR_float(Math.abs(x.get())); }
+ public static MR_double abs ( MR_double x ) { return new MR_double(Math.abs(x.get())); }
+
+ public static Inv inv ( MRData x ) { return new Inv(x); }
+
+ public static MR_bool and ( MR_bool x, MR_bool y ) { return (x.get()) ? y : false_value; }
+ public static MR_bool or ( MR_bool x, MR_bool y ) { return (x.get()) ? true_value : y; }
+ public static MR_bool not ( MR_bool x ) { return (x.get()) ? false_value : true_value; }
+
+ public static MR_bool toBool ( MR_string s ) { return (s.get().equals("true")) ? true_value : false_value; }
+ public static MR_short toShort ( MR_string s ) { return new MR_short(Short.parseShort(s.get())); }
+ public static MR_int toInt ( MR_string s ) { return new MR_int(Integer.parseInt(s.get())); }
+ public static MR_long toLong ( MR_string s ) { return new MR_long(Long.parseLong(s.get())); }
+ public static MR_float toFloat ( MR_string s ) { return new MR_float(Float.parseFloat(s.get())); }
+ public static MR_double toDouble ( MR_string s ) { return new MR_double(Double.parseDouble(s.get())); }
+
+ public static MR_int random ( MR_int n ) {
+ int v = random.nextInt(n.get());
+ return new MR_int(v);
+ }
+
+ public static MR_float log ( MR_float n ) { return new MR_float(Math.log(n.get())); }
+ public static MR_double log ( MR_double n ) { return new MR_double(Math.log(n.get())); }
+ public static MR_float exp ( MR_float n ) { return new MR_float(Math.exp(n.get())); }
+ public static MR_double exp ( MR_double n ) { return new MR_double(Math.exp(n.get())); }
+
+ public static MR_string string ( MRData x ) { return new MR_string(x.toString()); }
+
+ public static MR_bool contains ( MR_string x, MR_string y ) { return new MR_bool(x.get().contains(y.get())); }
+ public static MR_int length ( MR_string x ) { return new MR_int(x.get().length()); }
+ public static MR_string substring ( MR_string x, MR_int b, MR_int e ) { return new MR_string(x.get().substring(b.get(),e.get())); }
+
+ public static MR_bool exists ( Bag s ) {
+ return (s.iterator().hasNext()) ? true_value : false_value;
+ }
+
+ public static MR_bool some ( Bag x ) {
+ for ( MRData e: x )
+ if (e instanceof MR_bool)
+ if (((MR_bool)e).get())
+ return true_value;
+ return false_value;
+ }
+
+ public static MR_bool all ( Bag x ) {
+ for ( MRData e: x )
+ if (e instanceof MR_bool)
+ if (!((MR_bool)e).get())
+ return false_value;
+ return true_value;
+ }
+
+ public static MR_bool member ( MRData e, Bag s ) {
+ return (s.contains(e)) ? true_value : false_value;
+ }
+
+ public static MR_long count ( Bag s ) {
+ if (s.materialized())
+ return new MR_long(s.size());
+ long i = 0;
+ for ( MRData e: s )
+ i++;
+ return new MR_long(i);
+ }
+
+ public static MR_long hash_code ( MRData x ) {
+ return new MR_long(x.hashCode());
+ }
+
+ public static MRData index ( Bag b, MR_int mi ) {
+ int i = mi.get();
+ if (i < 0)
+ throw new Error("wrong index: "+i);
+ if (b.materialized())
+ return b.get(i);
+ int k = 0;
+ for ( MRData e: b )
+ if (k++ == i)
+ return e;
+ throw new Error("wrong index: "+i);
+ }
+
+ public static Bag range ( Bag b, MR_int mi, MR_int mj ) {
+ int i = mi.get();
+ int j = mj.get();
+ if (j < i)
+ throw new Error("wrong range indexes: "+i+","+j);
+ Bag bag = new Bag(j-i+1);
+ int k = 0;
+ for ( MRData e: b ) {
+ if (k >= i && k <= j)
+ bag.add(e);
+ k++;
+ };
+ return bag;
+ }
+
+ public static Bag union ( Bag x, Bag y ) {
+ return x.union(y);
+ }
+
+ public static Bag intersect ( Bag x, Bag y ) {
+ x.materialize();
+ Bag s = new Bag();
+ for ( MRData e: y )
+ if (x.contains(e))
+ s.add(e);
+ return s;
+ }
+
+ public static Bag except ( Bag x, Bag y ) {
+ y.materialize();
+ Bag s = new Bag();
+ for ( MRData e: x )
+ if (!y.contains(e))
+ s.add(e);
+ return s;
+ }
+
+ public static Bag materialize ( Bag x ) {
+ x.materialize();
+ return x;
+ }
+
+ /** coerce a basic type to a new type indicated by the basic type number */
+ public static MRData coerce ( MRData from, MR_int type ) {
+ byte tp = (byte)type.get();
+ if (from instanceof MR_short) {
+ if (tp == MRContainer.BYTE)
+ return new MR_byte((byte)((MR_short)from).get());
+ else if (tp == MRContainer.SHORT)
+ return from;
+ else if (tp == MRContainer.INT)
+ return new MR_int((int)((MR_short)from).get());
+ else if (tp == MRContainer.LONG)
+ return new MR_long((long)((MR_short)from).get());
+ else if (tp == MRContainer.FLOAT)
+ return new MR_float((float)((MR_short)from).get());
+ else if (tp == MRContainer.DOUBLE)
+ return new MR_double((double)((MR_short)from).get());
+ } else if (from instanceof MR_int) {
+ if (tp == MRContainer.BYTE)
+ return new MR_byte((byte)((MR_int)from).get());
+ else if (tp == MRContainer.SHORT)
+ return new MR_short((short)((MR_int)from).get());
+ else if (tp == MRContainer.INT)
+ return from;
+ else if (tp == MRContainer.LONG)
+ return new MR_long((long)((MR_int)from).get());
+ else if (tp == MRContainer.FLOAT)
+ return new MR_float((float)((MR_int)from).get());
+ else if (tp == MRContainer.DOUBLE)
+ return new MR_double((double)((MR_int)from).get());
+ } else if (from instanceof MR_long) {
+ if (tp == MRContainer.BYTE)
+ return new MR_byte((byte)((MR_long)from).get());
+ else if (tp == MRContainer.SHORT)
+ return new MR_short((short)((MR_long)from).get());
+ else if (tp == MRContainer.INT)
+ return new MR_int((int)((MR_long)from).get());
+ else if (tp == MRContainer.LONG)
+ return from;
+ else if (tp == MRContainer.FLOAT)
+ return new MR_float((float)((MR_long)from).get());
+ else if (tp == MRContainer.DOUBLE)
+ return new MR_double((double)((MR_long)from).get());
+ } else if (from instanceof MR_float) {
+ if (tp == MRContainer.BYTE)
+ return new MR_byte((byte)((MR_float)from).get());
+ else if (tp == MRContainer.SHORT)
+ return new MR_short((short)((MR_float)from).get());
+ else if (tp == MRContainer.INT)
+ return new MR_int((int)((MR_float)from).get());
+ else if (tp == MRContainer.LONG)
+ return new MR_long((long)((MR_float)from).get());
+ if (tp == MRContainer.FLOAT)
+ return from;
+ else if (tp == MRContainer.DOUBLE)
+ return new MR_double((double)((MR_float)from).get());
+ } else if (from instanceof MR_double) {
+ if (tp == MRContainer.BYTE)
+ return new MR_byte((byte)((MR_double)from).get());
+ else if (tp == MRContainer.SHORT)
+ return new MR_short((short)((MR_double)from).get());
+ else if (tp == MRContainer.INT)
+ return new MR_int((int)((MR_double)from).get());
+ else if (tp == MRContainer.LONG)
+ return new MR_long((long)((MR_double)from).get());
+ if (tp == MRContainer.FLOAT)
+ return new MR_float((float)((MR_double)from).get());
+ if (tp == MRContainer.DOUBLE)
+ return from;
+ };
+ error("Cannot up-coerce the numerical value "+from);
+ return null;
+ }
+
+ /** used in avg */
+ public static MR_double avg_value ( MRData t ) {
+ MR_double sum = (MR_double)((Tuple)t).first();
+ MR_long count = (MR_long)((Tuple)t).second();
+ return new MR_double(sum.get()/count.get());
+ }
+
+ public static MR_string text ( Union node ) {
+ if (node.tag() == 1)
+ return (MR_string)(node.value());
+ Bag b = (Bag)((Tuple)node.value()).get(2);
+ String s = "";
+ for ( MRData e: b )
+ if (((Union)e).tag() == 1)
+ s += ((MR_string)(((Union)e).value())).get();
+ return new MR_string(s);
+ }
+
+ public static MR_string text ( Bag nodes ) {
+ MR_string b = new MR_string("");
+ for ( MRData e: nodes )
+ b = plus(b,text((Union)e));
+ return b;
+ }
+
+ public static MR_string tag ( Union node ) {
+ if (node.tag() == 1)
+ error("Cannot extract the tagname of a CData: "+node);
+ return (MR_string)((Tuple) node.value()).get(0);
+ }
+
+ public static MR_string XMLattribute ( MR_string tagname, Union node ) {
+ if (node.tag() == 1)
+ error("Element "+node+" does not have attributes");
+ Tuple t = (Tuple)node.value();
+ String tag = tagname.get();
+ for ( MRData c: (Bag)t.get(1) ) {
+ Tuple p = (Tuple)c;
+ if (tag.equals(((MR_string)(p.get(0))).get()))
+ return new MR_string(((MR_string)p.get(1)).get());
+ };
+ error("Element "+node+" does not have attribute "+tagname);
+ return null;
+ }
+
+ public static Bag XMLattributes ( MR_string tagname, Union node ) {
+ if (node.tag() == 1)
+ return new Bag();
+ Tuple t = (Tuple)node.value();
+ Bag b = new Bag();
+ String tag = tagname.get();
+ for ( MRData c: (Bag)t.get(1) ) {
+ Tuple p = (Tuple)c;
+ if (tag.equals("*") || tag.equals(((MR_string)(p.get(0))).get()))
+ b.add(p.get(1));
+ };
+ return b;
+ }
+
+ public static Bag XMLattributes ( MR_string tagname, Bag nodes ) {
+ Bag b = new Bag();
+ for ( MRData e: nodes )
+ for (MRData c: XMLattributes(tagname,(Union)e))
+ b.add(c);
+ return b;
+ }
+
+ public static Bag XMLattribute ( MR_string tagname, Bag nodes ) {
+ Bag b = new Bag();
+ for ( MRData e: nodes )
+ for (MRData c: XMLattributes(tagname,(Union)e))
+ b.add(c);
+ return b;
+ }
+
+ public static Bag XMLchildren ( MR_string tagname, Union node ) {
+ if (node.tag() == 1)
+ return new Bag();
+ Tuple t = (Tuple)node.value();
+ Bag b = new Bag();
+ String tag = tagname.get();
+ for ( MRData c: (Bag)t.get(2) )
+ if (((Union)c).tag() == 0) {
+ Tuple s = (Tuple)(((Union)c).value());
+ if (tag.equals("*") || (((MR_string)(s.get(0))).get()).equals(tag))
+ b.add(c);
+ };
+ return b;
+ }
+
+ public static Bag XMLchildren ( MR_string tagname, Bag nodes ) {
+ Bag b = new Bag();
+ for ( MRData e: nodes )
+ for (MRData c: XMLchildren(tagname,(Union)e))
+ b.add(c);
+ return b;
+ }
+
+ public static MRData fold ( Lambda c, MRData z, Bag s ) {
+ MRData v = z;
+ for ( MRData e: s )
+ z = c.lambda().eval(new Tuple(z,e));
+ return z;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Test.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Test.java b/core/src/main/java/org/apache/mrql/Test.java
new file mode 100644
index 0000000..eca272c
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Test.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.util.*;
+import org.apache.hadoop.conf.*;
+
+
+/** Test all the MRQL test queries */
+final public class Test {
+ public static PrintStream print_stream;
+ public static Configuration conf;
+ static MRQLParser parser = new MRQLParser();
+ static String result_directory;
+ static PrintStream test_out;
+ static PrintStream error_stream;
+
+ private static int compare ( String file1, String file2 ) throws Exception {
+ FileInputStream s1 = new FileInputStream(file1);
+ FileInputStream s2 = new FileInputStream(file2);
+ int b1, b2;
+ int i = 1;
+ while ((b1 = s1.read()) == (b2 = s2.read()) && b1 != -1 && b2 != -1)
+ i++;
+ return (b1 == -1 && b2 == -1) ? 0 : i;
+ }
+
+ private static void query ( File query ) throws Exception {
+ String path = query.getPath();
+ if (!path.endsWith(".mrql"))
+ return;
+ Translator.global_reset();
+ String qname = query.getName();
+ qname = qname.substring(0,qname.length()-5);
+ test_out.print(" Testing "+qname+" ... ");
+ String result_file = result_directory+"/"+qname+".txt";
+ boolean exists = new File(result_file).exists();
+ if (exists)
+ System.setOut(new PrintStream(result_directory+"/result.txt"));
+ else System.setOut(new PrintStream(result_file));
+ try {
+ parser = new MRQLParser(new MRQLLex(new FileInputStream(query)));
+ Main.parser = parser;
+ MRQLLex.reset();
+ parser.parse();
+ int i;
+ if (exists && (i = compare(result_file,result_directory+"/result.txt")) > 0)
+ test_out.println("MISMATCH AT "+(i-1));
+ else if (exists)
+ test_out.println("OK matched");
+ else test_out.println("OK created");
+ } catch (Error ex) {
+ error_stream.println(qname+": "+ex);
+ ex.printStackTrace(error_stream);
+ test_out.println("FAILED");
+ if (!exists)
+ new File(result_file).delete();
+ } catch (Exception ex) {
+ error_stream.println(qname+": "+ex);
+ ex.printStackTrace(error_stream);
+ test_out.println("FAILED");
+ if (!exists)
+ new File(result_file).delete();
+ } finally {
+ if (Config.hadoop_mode)
+ Plan.clean();
+ if (Config.compile_functional_arguments)
+ Compiler.clean();
+ }
+ }
+
+ public static void main ( String[] args ) throws Exception {
+ boolean hadoop = false;
+ for ( String arg: args ) {
+ hadoop |= arg.equals("-local") || arg.equals("-dist");
+ Config.bsp_mode |= arg.equals("-bsp");
+ Config.spark_mode |= arg.equals("-spark");
+ };
+ Config.map_reduce_mode = !Config.bsp_mode && !Config.spark_mode;
+ if (Config.map_reduce_mode)
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.MapReduceEvaluator").newInstance();
+ if (Config.bsp_mode)
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.BSPEvaluator").newInstance();
+ if (Config.spark_mode)
+ Evaluator.evaluator = (Evaluator)Class.forName("org.apache.mrql.SparkEvaluator").newInstance();
+ Config.quiet_execution = true;
+ if (hadoop) {
+ conf = Evaluator.evaluator.new_configuration();
+ GenericOptionsParser gop = new GenericOptionsParser(conf,args);
+ conf = gop.getConfiguration();
+ args = gop.getRemainingArgs();
+ };
+ Config.parse_args(args,conf);
+ Config.hadoop_mode = Config.local_mode || Config.distributed_mode;
+ Evaluator.evaluator.init(conf);
+ new TopLevel();
+ Config.testing = true;
+ if (Config.hadoop_mode && Config.bsp_mode)
+ Config.write(Plan.conf);
+ if (Main.query_file.equals("") || Config.extra_args.size() != 2)
+ throw new Error("Must provide a query directory, a result directory, and an error log file");
+ File query_dir = new File(Main.query_file);
+ result_directory = Config.extra_args.get(0);
+ (new File(result_directory)).mkdirs();
+ error_stream = new PrintStream(Config.extra_args.get(1));
+ System.setErr(error_stream);
+ test_out = System.out;
+ for ( File f: query_dir.listFiles() )
+ query(f);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/TopLevel.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/TopLevel.gen b/core/src/main/java/org/apache/mrql/TopLevel.gen
new file mode 100644
index 0000000..0ed3f49
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/TopLevel.gen
@@ -0,0 +1,293 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+
+
+/** Provides the API for compilation/code-generation */
+final public class TopLevel extends Interpreter {
+ static Tree xml_type;
+
+ public TopLevel () {
+ // XML and JSON are user-defined types:
+ datadef("XML",#<union(Node(tuple(string,bag(tuple(string,string)),list(XML))),
+ CData(string))>);
+ datadef("JSON",#<union(JObject(bag(tuple(string,JSON))),
+ JArray(list(JSON)),
+ Jstring(string),
+ Jlong(long),
+ Jdouble(double),
+ Jbool(bool),
+ Jnull(tuple()))>);
+ constant(#<PI>,#<double>,new MR_double(Math.PI));
+ xml_type = global_datatype_env.lookup("XML");
+ DataSource.loadParsers();
+ }
+
+ private static boolean memory_parsed_source ( Tree e ) {
+ match e {
+ case ParsedSource(...): return true;
+ case Merge(`x,`y): return memory_parsed_source(x) && memory_parsed_source(y);
+ };
+ return false;
+ }
+
+ /** translate and evaluate an MRQL expression into MRData
+ * @param e MRQL query to be evaluated
+ * @param print do we want to print the result?
+ * @return the result of evaluation (MRData)
+ */
+ public static MRData expression ( Tree e, boolean print ) {
+ try {
+ Tree plan = translate_expression(e);
+ query_plan = plan;
+ tab_count = -3;
+ if (plan == null)
+ return null;
+ if (Config.hadoop_mode)
+ Evaluator.evaluator.initialize_query();
+ MRData res = evalE(plan,null);
+ if (print) {
+ if (!Config.quiet_execution)
+ System.out.println("Result:");
+ if (!Config.hadoop_mode && Config.bsp_mode && memory_parsed_source(plan))
+ System.out.println(print(((Tuple)((Bag)res).get(0)).second(),query_type));
+ else System.out.println(print(res,query_type));
+ } return res;
+ } catch (Exception x) {
+ if (x.getMessage() != null)
+ System.err.println("*** MRQL System Error at line "+Main.parser.line_pos()+": "+x);
+ if (Config.trace)
+ x.printStackTrace(System.err);
+ if (Config.testing)
+ throw new Error(x);
+ return null;
+ } catch (Error x) {
+ if (x.getMessage() != null)
+ System.err.println("*** MRQL System Error at line "+Main.parser.line_pos()+": "+x);
+ if (Config.trace)
+ x.printStackTrace(System.err);
+ if (Config.testing)
+ throw new Error(x);
+ return null;
+ }
+ }
+
+ /** translate, evaluate, and print the results of an MRQL expression e
+ * @param e MRQL query to be evaluated
+ * @return the result of evaluation (MRData)
+ */
+ public final static MRData expression ( Tree e ) {
+ reset();
+ return expression(e,true);
+ }
+
+ /** handle the assignment v=e */
+ public final static void assign ( String v, Tree e ) {
+ if (variable_lookup(v,global_env) != null) {
+ global_type_env.remove(v);
+ remove_global_binding(v);
+ };
+ global_vars.insert(v,e);
+ }
+
+ private final static boolean is_function ( Tree e ) {
+ match e {
+ case function(...): return true;
+ };
+ return false;
+ }
+
+ /** handle the assignment v:=e */
+ public final static Tree store ( String v, Tree e ) {
+ reset();
+ if (global_vars.lookup(v) != null)
+ global_vars.remove(v);
+ MRData res = expression(e,false);
+ global_type_env.insert(v,query_type);
+ if (res instanceof Bag)
+ ((Bag)res).materialize();
+ new_global_binding(v,res);
+ return query_plan;
+ }
+
+ /** define an MRQL constant, such as PI */
+ private final static void constant ( Tree v, Tree type, MRData value ) {
+ String var = v.toString();
+ if (global_vars.lookup(var) != null)
+ global_vars.remove(var);
+ global_type_env.insert(var,type);
+ new_global_binding(var,value);
+ }
+
+ /** define a new function
+ * @param fnc function name
+ * @param params parameter list
+ * @param out_type output type
+ * @param body function body
+ */
+ public final static void functiondef ( String fnc, Trees params, Tree out_type, Tree body ) {
+ reset();
+ Trees as = #[];
+ Trees ps = #[];
+ for ( Tree param: params )
+ match param {
+ case bind(`v,`tp):
+ Tree ntp = normalize_type(tp);
+ as = as.append(ntp);
+ ps = ps.append(#<bind(`v,`ntp)>);
+ case _: type_error(param,"Ill-formed function parameter: "+param);
+ };
+ out_type = normalize_type(out_type);
+ // needed for recursive functions
+ global_type_env.insert(fnc,#<arrow(tuple(...as),`out_type)>);
+ Tree fname = #<`fnc>;
+ if (!is_pure(body))
+ impure_functions = impure_functions.append(fname);
+ Tree plan = store(fnc,#<function(tuple(...ps),`out_type,`body)>);
+ if (plan != null)
+ Translator.global_functions.insert(fnc,plan);
+ if (Config.hadoop_mode && plan != null)
+ Plan.conf.set("mrql.global."+fnc,
+ closure(plan,global_env).toString());
+ }
+
+ /** dump the result of evaluating the MRQL query e to a binary file */
+ private final static void dump ( String file, Tree e ) {
+ MRData res = expression(e,false);
+ try {
+ query_type = make_persistent_type(query_type);
+ if (res != null)
+ if (Config.hadoop_mode)
+ Evaluator.evaluator.dump(file,query_type,res);
+ else MapReduceAlgebra.dump(file,query_type,res);
+ } catch (Exception x) {
+ throw new Error(x);
+ }
+ }
+
+ /** dump the result of evaluating the MRQL query e to a text CVS file */
+ private final static void dump_text ( String file, Tree e ) {
+ MRData res = expression(e,false);
+ if (res != null)
+ try {
+ Evaluator.evaluator.dump_text(file,query_type,res);
+ } catch (Exception x) {
+ throw new Error(x);
+ }
+ }
+
+ /** define a new named type (typedef) */
+ private final static void typedef ( String name, Tree type ) {
+ type_names.insert(name,normalize_type(type));
+ }
+
+ /** define a new data type, such as XML and JSON */
+ private final static void datadef ( String name, Tree type ) {
+ int i = 0;
+ Trees as = #[];
+ match type {
+ case union(...nl):
+ // needed for recursive datatypes
+ global_datatype_env.insert(name,#<union(...nl)>);
+ for ( Tree n: nl )
+ match n {
+ case `c(`t):
+ if (data_constructors.lookup(c.toString()) == null)
+ data_constructors.insert(c.toString(),#<`name(`i,`t)>);
+ else type_error(type,"Data constructor "+c+" has already been defined");
+ as = as.append(#<`c(`(normalize_type(t)))>);
+ i++;
+ }
+ };
+ global_datatype_env.remove(name);
+ global_datatype_env.insert(name,#<union(...as)>);
+ }
+
+ /** define a user aggregation */
+ private static void aggregation ( String name, Tree type, Tree plus, Tree zero, Tree unit ) {
+ reset();
+ zero = Simplification.rename(zero);
+ plus = Simplification.rename(plus);
+ unit = Simplification.rename(unit);
+ type = normalize_type(type);
+ Tree ztp = TypeInference.type_inference2(zero);
+ Tree v1 = new_var();
+ type_env.insert(v1.toString(),ztp);
+ TypeInference.type_inference2(Normalization.normalize_all(#<apply(`plus,tuple(`v1,`v1))>));
+ Tree v2 = new_var();
+ type_env.insert(v2.toString(),type);
+ Tree utp = TypeInference.type_inference2(Normalization.normalize_all(#<apply(`unit,`v2)>));
+ if (unify(utp,ztp) == null)
+ type_error(unit,"Wrong type in unit result (expected "+ztp+" found "+utp);
+ monoids = monoids.append(#<`name(`type,`plus,`zero,`unit)>);
+ }
+
+ /** the MRQL top-level interfacse to evaluate a single MRQL expression or command */
+ public final static void evaluate_top_level ( Tree expr ) {
+ if (expr == null)
+ return;
+ match expr {
+ case expression(`e):
+ long t = System.currentTimeMillis();
+ if (expression(e) != null && !Config.quiet_execution)
+ System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
+ case assign(`v,`e): assign(v.toString(),e);
+ case store(`v,`e):
+ long t = System.currentTimeMillis();
+ if (store(v.toString(),e) != null && !Config.quiet_execution)
+ System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
+ case dump(`s,`e):
+ long t = System.currentTimeMillis();
+ dump(s.stringValue(),e);
+ if (!Config.quiet_execution)
+ System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
+ case dump_text(`s,`e):
+ long t = System.currentTimeMillis();
+ dump_text(s.stringValue(),e);
+ if (!Config.quiet_execution)
+ System.out.println("Run time: "+(System.currentTimeMillis()-t)/1000.0+" secs");
+ case typedef(`v,`t): typedef(v.toString(),t);
+ case datadef(`v,`t): datadef(v.toString(),t);
+ case functiondef(`f,params(...p),`tp,`e):
+ functiondef(f.toString(),p,tp,e);
+ case macrodef(`name,params(...p),`e):
+ Translator.global_macros.insert(name.toString(),#<macro(params(...p),`e)>);
+ case aggregation(`aggr,`type,`plus,`zero,`unit):
+ aggregation(aggr.toString(),type,plus,zero,unit);
+ case import(`c):
+ ClassImporter.importClass(c.variableValue());
+ case import(`c,...l):
+ for (Tree m: l)
+ ClassImporter.importMethod(c.variableValue(),m.variableValue());
+ case include(`file):
+ Main.include_file(file.toString());
+ case parser(`n,`p):
+ try {
+ Class<? extends Parser> c = Class.forName(p.toString()).asSubclass(Parser.class);
+ DataSource.parserDirectory.put(n.toString(),c);
+ } catch (ClassNotFoundException e) {
+ throw new Error("Class "+p.toString()+" not found");
+ }
+ case impure(`fn): // not used
+ impure_functions = impure_functions.append(fn);
+ case _: throw new Error("Unknown statement: "+expr);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Translator.gen
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Translator.gen b/core/src/main/java/org/apache/mrql/Translator.gen
new file mode 100644
index 0000000..47b34ac
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Translator.gen
@@ -0,0 +1,445 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import org.apache.mrql.gen.*;
+import java.util.*;
+import java.io.*;
+
+
+/** contains useful methods used by all stages of compilation and code generation */
+public class Translator extends Printer {
+ static Trees functions = #[ ];
+
+ static {
+ ClassImporter.load_classes();
+ DataSource.loadParsers();
+ }
+
+ /** type environment that binds local variables to types */
+ static SymbolTable type_env = new SymbolTable();
+
+ /** type environment that binds global variables to types */
+ static SymbolTable global_type_env = new SymbolTable();
+
+ /** type environment that binds datatype names to types */
+ static SymbolTable global_datatype_env = new SymbolTable();
+
+ /** used in pattern compilation and variable renaming */
+ static SymbolTable st = new SymbolTable();
+
+ /** binds macro names to MRQL expressions (used for 'name = expr;' syntax) */
+ static SymbolTable global_vars = new SymbolTable();
+
+ /** binds a UDF name to its plan */
+ static SymbolTable global_functions = new SymbolTable();
+
+ /** binds a macro name to its body */
+ static SymbolTable global_macros = new SymbolTable();
+
+ /** used in typedefs */
+ static SymbolTable type_names = new SymbolTable();
+
+ /** binds a data constructor name to its type */
+ static SymbolTable data_constructors = new SymbolTable();
+
+ static Trees repeat_variables = #[];
+
+ static {
+ global_type_env.insert("args",#<list(string)>);
+ }
+
+ /** expressions with impure functions cannot factored out */
+ static Trees impure_functions = #[random];
+
+ private static int var_count = 0;
+
+ static void reset () {
+ var_count = 0;
+ type_env = global_type_env;
+ st = new SymbolTable();
+ type_env = new SymbolTable();
+ repeat_variables = #[];
+ }
+
+ static void global_reset () {
+ reset();
+ global_type_env = new SymbolTable();
+ global_datatype_env = new SymbolTable();
+ global_vars = new SymbolTable();
+ global_functions = new SymbolTable();
+ global_macros = new SymbolTable();
+ type_names = new SymbolTable();
+ data_constructors = new SymbolTable();
+ new TopLevel();
+ }
+
+ static void error ( String msg ) {
+ System.err.println("*** MRQL error at line "+Main.parser.line_pos()+": "+msg);
+ throw new Error();
+ }
+
+ final static Tree identity = #<lambda(x,x)>;
+
+ static Tree identity () {
+ return Normalization.rename(#<lambda(x,bag(x))>);
+ }
+
+ /** is this type a collection type? */
+ public static boolean is_collection ( String x ) {
+ return x.equals("Bag") || x.equals("bag") || x.equals("List") || x.equals("list");
+ }
+
+ /** is this type a collection type? */
+ public static boolean collection_type ( Tree tp ) {
+ match tp {
+ case `T(`t1): return is_collection(T);
+ };
+ return false;
+ }
+
+ /** is this type a collection type for values stored in HDFS? */
+ public static boolean is_persistent_collection ( String x ) {
+ return x.equals("Bag") || x.equals("List");
+ }
+
+ /** make this collection type a persistent type that is stored in HDFS */
+ public static String persistent_collection ( String x ) {
+ return (x.equals("list")) ? "List" : (x.equals("bag")) ? "Bag" : x;
+ }
+
+ /** make this collection type a transient type stored in memory */
+ public static String transient_collection ( String x ) {
+ return (x.equals("List")) ? "list" : (x.equals("Bag")) ? "bag" : x;
+ }
+
+ /** An aggeregation must be based on a commutative monoid (plus,zero) with a unit:
+ * name(type,plus,zero,unit)
+ * plus: function from (b,b) to b, zero: b, unit: function from a to b
+ */
+ static Trees monoids =
+ #[ count(any,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0,long),lambda(x,typed(1,long))),
+ sum(int,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0,long),lambda(x,typed(x,long))),
+ sum(long,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0,long),`identity),
+ sum(float,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0.0,double),lambda(x,typed(x,double))),
+ sum(double,lambda(x,call(plus,nth(x,0),nth(x,1))),typed(0.0,double),`identity),
+ max(int,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Integer.MIN_VALUE),int),`identity),
+ max(long,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Long.MIN_VALUE),long),`identity),
+ max(float,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Float.MIN_VALUE),float),`identity),
+ max(double,lambda(x,call(max,nth(x,0),nth(x,1))),typed(`(Double.MIN_VALUE),double),`identity),
+ min(int,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Integer.MAX_VALUE),int),`identity),
+ min(long,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Long.MAX_VALUE),long),`identity),
+ min(float,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Float.MAX_VALUE),float),`identity),
+ min(double,lambda(x,call(min,nth(x,0),nth(x,1))),typed(`(Double.MAX_VALUE),double),`identity),
+ avg_aggr(int,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
+ call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
+ tuple(typed(0.0,double),typed(0,long)),
+ lambda(x,tuple(typed(x,double),typed(1,long)))),
+ avg_aggr(long,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
+ call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
+ tuple(typed(0.0,double),typed(0,long)),
+ lambda(x,tuple(typed(x,double),typed(1,long)))),
+ avg_aggr(float,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
+ call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
+ tuple(typed(0.0,double),typed(0,long)),
+ lambda(x,tuple(typed(x,double),typed(1,long)))),
+ avg_aggr(double,lambda(x,tuple(call(plus,nth(nth(x,0),0),nth(nth(x,1),0)),
+ call(plus,nth(nth(x,0),1),nth(nth(x,1),1)))),
+ tuple(typed(0.0,double),typed(0,long)),
+ lambda(x,tuple(typed(x,double),typed(1,long)))),
+ all(bool,lambda(x,call(and,nth(x,0),nth(x,1))),true,`identity),
+ some(bool,lambda(x,call(or,nth(x,0),nth(x,1))),false,`identity)
+ ];
+
+ static void print_aggregates () {
+ for ( Tree m: monoids )
+ match m {
+ case `f(`tp,...):
+ System.out.print(" "+f+":"+print_type(tp));
+ }
+ System.out.println();
+ }
+
+ static Trees plans_with_distributed_lambdas
+ = #[MapReduce,MapAggregateReduce,MapCombineReduce,FroupByJoin,Aggregate,
+ MapReduce2,MapCombineReduce2,MapAggregateReduce2,MapJoin,MapAggregateJoin,
+ CrossProduct,CrossAggregateProduct,cMap,AggregateMap,BSP,GroupByJoin,repeat,closure];
+
+ static Trees algebraic_operators
+ = #[mapReduce,mapReduce2,cmap,join,groupBy,orderBy,aggregate,map,filter];
+
+ static Trees plan_names = plans_with_distributed_lambdas.append(algebraic_operators)
+ .append(#[Repeat,Closure,Generator,Let,If]);
+
+ /** generates new variable names */
+ public static Tree new_var () {
+ return new VariableLeaf("x_"+(Integer.toString(var_count++)));
+ }
+
+ /** is this expression pure? (does it contain calls to impure functions?) */
+ static boolean is_pure ( Tree expr ) {
+ match expr {
+ case call(`f,...al):
+ if (impure_functions.member(f))
+ return false;
+ else fail
+ case `f(...al):
+ for ( Tree a: al )
+ if (!is_pure(a))
+ return false;
+ };
+ return true;
+ }
+
+ public static Trees union ( Trees xs, Trees ys ) {
+ Trees res = xs;
+ for ( Tree y: ys )
+ if (!xs.member(y))
+ res = res.append(y);
+ return res;
+ }
+
+ /** return the variables of a pattern */
+ static Trees pattern_variables ( Tree pattern ) {
+ Trees args = #[];
+ match pattern {
+ case tuple(...pl):
+ for ( Tree p: pl )
+ args = union(args,pattern_variables(p));
+ case record(...bl):
+ for ( Tree b: bl )
+ match b {
+ case bind(`n,`p):
+ args = union(args,pattern_variables(p));
+ };
+ case typed(`p,_):
+ args = pattern_variables(p);
+ case `v:
+ if (v.is_variable())
+ args = #[`v];
+ };
+ return args;
+ }
+
+ /** replace all occurences of from_expr in expr with to_expr
+ * @param from_expr target
+ * @param to_expr replacement
+ * @param expr input
+ * @return equal to expr but with all occurences of from_expr replaced with to_expr
+ */
+ public static Tree subst ( Tree from_expr, Tree to_expr, Tree expr ) {
+ if (expr.equals(from_expr))
+ return to_expr;
+ match expr {
+ case lambda(`v,_):
+ if (pattern_variables(v).member(from_expr))
+ return expr;
+ else fail
+ case bind(`a,`u):
+ return #<bind(`a,`(subst(from_expr,to_expr,u)))>;
+ case `f(...al):
+ return #<`f(...(subst_list(from_expr,to_expr,al)))>;
+ };
+ return expr;
+ }
+
+ /** replace all occurences of from_expr in el with to_expr
+ * @param from_expr target
+ * @param to_expr replacement
+ * @param el list of input expressions
+ * @return equal to el but with all occurences of from_expr replaced with to_expr
+ */
+ public static Trees subst_list ( Tree from_expr, Tree to_expr, Trees el ) {
+ Trees bl = #[];
+ for ( Tree e: el )
+ bl = bl.append(subst(from_expr,to_expr,e));
+ return bl;
+ }
+
+ /** replace all occurences of var in expr with to_expr only if to_expr is pure or it is used once only
+ * @param var target
+ * @param to_expr replacement
+ * @param expr input
+ * @return equal to expr but with all occurences of from_expr replaced with to_expr
+ */
+ public static Tree subst_var ( Tree var, Tree to_expr, Tree expr ) {
+ if (!is_pure(to_expr) && occurences(var,expr) > 1)
+ return #<let(`var,`to_expr,`expr)>;
+ else return subst(var,to_expr,expr);
+ }
+
+ /** used in the MRQL parser to handle templates */
+ public static Tree template ( Tree s ) {
+ match s {
+ case template(`parser,...as):
+ try {
+ Trees args = #[];
+ String tx = "";
+ int i = 0;
+ for ( Tree a: as )
+ match a {
+ case text(`t): tx += t;
+ case _: args = args.append(a);
+ tx += "{{"+(i++)+"}}";
+ };
+ Class<? extends Parser> pc = DataSource.parserDirectory.get(parser.toString());
+ if (pc == null)
+ throw new Error("Unrecognized parser: "+parser);
+ Parser p = pc.newInstance();
+ p.initialize(#[]);
+ Bag e = p.parse(tx);
+ Tree res = Interpreter.reify(e.get(0),p.type());
+ for ( int j = 0; j < i; j++ )
+ res = subst(new VariableLeaf("t_"+j),args.nth(j),res);
+ return res;
+ } catch (Exception e) {
+ throw new Error("Wrong template: "+s+"\n"+e);
+ }
+ };
+ throw new Error("Wrong template: "+s);
+ }
+
+ /** convert Tree constructions to code that construct these Trees (used in the Compiler) */
+ public static String reify ( Tree e ) {
+ if (e instanceof LongLeaf)
+ return "new org.apache.mrql.gen.LongLeaf(" + e + ")";
+ else if (e instanceof DoubleLeaf)
+ return "new org.apache.mrql.gen.DoubleLeaf(" + e + ")";
+ else if (e instanceof VariableLeaf)
+ return "new org.apache.mrql.gen.VariableLeaf(\"" + e.variableValue() + "\")";
+ else if (e instanceof StringLeaf)
+ return "new org.apache.mrql.gen.StringLeaf(" + e.toString().replace("\\","\\\\") + ")";
+ else {
+ Node n = (Node) e;
+ return "new org.apache.mrql.gen.Node(\""+n.name()+"\","+reify(n.children())+")";
+ }
+ }
+
+ /** convert Tree constructions to code that construct these Trees (used in the Compiler) */
+ public static String reify ( Trees ts ) {
+ String s = "org.apache.mrql.gen.Trees.nil";
+ for ( Tree c: ts )
+ s += ".append("+reify(c)+")";
+ return s;
+ }
+
+ /** return the list of free variables in e that are not in exclude list */
+ public static Trees free_variables ( Tree e, Trees exclude ) {
+ if (e == null)
+ return #[];
+ match e {
+ case lambda(`x,`b):
+ return free_variables(b,exclude.append(pattern_variables(x)));
+ case let(`x,`u,`b):
+ return free_variables(b,exclude.append(pattern_variables(x)))
+ .append(free_variables(u,exclude));
+ case Let(`x,`u,`b):
+ return free_variables(b,exclude.append(pattern_variables(x)))
+ .append(free_variables(u,exclude));
+ case select(`u,from(...bs),`p):
+ Trees ex = exclude;
+ Trees fs = #[];
+ for ( Tree b: bs )
+ match b {
+ case bind(`v,`x):
+ fs = fs.append(free_variables(x,ex));
+ ex = ex.append(pattern_variables(v));
+ };
+ return free_variables(p,ex).append(free_variables(u,ex)).append(fs);
+ case `f(...as):
+ Trees res = #[];
+ for ( Tree a: as )
+ res = res.append(free_variables(a,exclude));
+ return res;
+ case `v:
+ if (v.is_variable() && v.toString().startsWith("x_") && !exclude.member(v))
+ return #[`v];
+ };
+ return #[];
+ }
+
+ /** count the occurences of x in e */
+ public static int occurences ( Tree x, Tree e ) {
+ if (x.equals(e))
+ return 1;
+ match e {
+ case `f(...as):
+ int i = 0;
+ for ( Tree a: as )
+ i += occurences(x,a);
+ return i;
+ };
+ return 0;
+ }
+
+ /** return true if x is equal to y modulo variable substitution */
+ public static boolean alpha_equivalent ( Tree x, Tree y, SymbolTable st ) {
+ match #<T(`x,`y)> {
+ case T(lambda(`vx,`bx),lambda(`vy,`by)):
+ if (!vx.equals(vy))
+ st.insert(vx.toString(),vy);
+ return alpha_equivalent(bx,by,st);
+ case T(`f(...xs),`g(...ys)):
+ if (!f.equals(g) || xs.length() != ys.length())
+ return false;
+ for ( ; !xs.is_empty(); xs = xs.tail(), ys = ys.tail() )
+ if (!alpha_equivalent(xs.head(),ys.head(),st))
+ return false;
+ return true;
+ case T(`v,`w):
+ if (v.is_variable() && w.is_variable())
+ return v.equals(w) || (st.lookup(v.toString()) != null
+ && st.lookup(v.toString()).equals(w));
+ };
+ return x.equals(y);
+ }
+
+ private static SymbolTable alpha_symbol_table = new SymbolTable();
+
+ /** return true if x is equal to y modulo variable substitution */
+ public static boolean alpha_equivalent ( Tree x, Tree y ) {
+ alpha_symbol_table.begin_scope();
+ boolean b = alpha_equivalent(x,y,alpha_symbol_table);
+ alpha_symbol_table.end_scope();
+ return b;
+ }
+
+ /** translate a simplified select MRQL query to an algebraic form */
+ public static Tree translate_select ( Tree e ) {
+ match e {
+ case select(`u,from(),where(true)):
+ return #<bag(`(translate_select(u)))>;
+ case select(`u,from(),where(`c)):
+ return #<if(`(translate_select(c)),bag(`(translate_select(u))),bag())>;
+ case select(`u,from(bind(`v,`d),...bl),where(`c)):
+ Tree n = translate_select(#<select(`u,from(...bl),where(`c))>);
+ return #<cmap(lambda(`v,`n),`(translate_select(d)))>;
+ case `f(...al):
+ Trees bl = #[];
+ for ( Tree a: al )
+ bl = bl.append(translate_select(a));
+ return #<`f(...bl)>;
+ };
+ return e;
+ }
+
+ /** the MRQL top-level interfacse to evaluate a single MRQL expression or command */
+ public static void top_level ( Tree expr ) {
+ TopLevel.evaluate_top_level(expr);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/core/src/main/java/org/apache/mrql/Tuple.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mrql/Tuple.java b/core/src/main/java/org/apache/mrql/Tuple.java
new file mode 100644
index 0000000..bced3dd
--- /dev/null
+++ b/core/src/main/java/org/apache/mrql/Tuple.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.mrql;
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+
+
+/** a container for Tuples */
+final public class Tuple extends MRData {
+ private final static long serialVersionUID = 723385754575L;
+
+ MRData[] tuple;
+
+ public Tuple ( int size ) {
+ tuple = new MRData[size];
+ }
+
+ public Tuple ( final MRData ...as ) {
+ tuple = as;
+ }
+
+ /** the number of elements in the tuple */
+ public short size () { return (short)tuple.length; }
+
+ public void materializeAll () {
+ for (MRData e: tuple)
+ e.materializeAll();
+ };
+
+ /** the i'th element of the tuple */
+ public MRData get ( int i ) {
+ return tuple[i];
+ }
+
+ /** the first element of the tuple */
+ public MRData first () { return tuple[0]; }
+
+ /** the second element of the tuple */
+ public MRData second () { return tuple[1]; }
+
+ /** replace the i'th element of a tuple with new data and return a new value */
+ public MRData set ( int i, MRData data, MRData ret ) {
+ tuple[i] = data;
+ return ret;
+ }
+
+ /** replace the i'th element of a tuple with new data */
+ public Tuple set ( int i, MRData data ) {
+ tuple[i] = data;
+ return this;
+ }
+
+ final public void write ( DataOutput out ) throws IOException {
+ if (tuple.length == 0)
+ out.writeByte(MRContainer.NULL);
+ else if (tuple.length == 2) {
+ out.writeByte(MRContainer.PAIR);
+ tuple[0].write(out);
+ tuple[1].write(out);
+ } else if (tuple.length == 3) {
+ out.writeByte(MRContainer.TRIPLE);
+ tuple[0].write(out);
+ tuple[1].write(out);
+ tuple[2].write(out);
+ } else {
+ out.writeByte(MRContainer.TUPLE);
+ WritableUtils.writeVInt(out,tuple.length);
+ for (short i = 0; i < tuple.length; i++)
+ tuple[i].write(out);
+ }
+ }
+
+ final public static Tuple read ( DataInput in ) throws IOException {
+ int n = WritableUtils.readVInt(in);
+ Tuple t = new Tuple(n);
+ for ( short i = 0; i < n; i++ )
+ t.tuple[i] = MRContainer.read(in);
+ return t;
+ }
+
+ final public static Tuple read2 ( DataInput in ) throws IOException {
+ return new Tuple(MRContainer.read(in),MRContainer.read(in));
+ }
+
+ final public static Tuple read3 ( DataInput in ) throws IOException {
+ return new Tuple(MRContainer.read(in),MRContainer.read(in),MRContainer.read(in));
+ }
+
+ public void readFields ( DataInput in ) throws IOException {
+ int n = WritableUtils.readVInt(in);
+ tuple = new Tuple[n];
+ for ( short i = 0; i < n; i++ )
+ tuple[i] = MRContainer.read(in);
+ }
+
+ public int compareTo ( MRData x ) {
+ assert(x instanceof Tuple);
+ Tuple t = (Tuple) x;
+ for ( short i = 0; i < tuple.length && i < t.tuple.length; i++ ) {
+ int c = get(i).compareTo(t.get(i));
+ if (c < 0)
+ return -1;
+ else if (c > 0)
+ return 1;
+ };
+ if (tuple.length > t.tuple.length)
+ return 1;
+ else if (tuple.length < t.tuple.length)
+ return -1;
+ else return 0;
+ }
+
+ final public static int compare ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ try {
+ int n = WritableComparator.readVInt(x,xs);
+ int s = WritableUtils.decodeVIntSize(x[xs]);
+ for ( short i = 0; i < n; i++ ) {
+ int k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
+ if (k != 0)
+ return k;
+ s += size[0];
+ };
+ size[0] = s+1;
+ return 0;
+ } catch (IOException e) {
+ throw new Error(e);
+ }
+ }
+
+ final public static int compare2 ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ int k = MRContainer.compare(x,xs,xl,y,ys,yl,size);
+ if (k != 0)
+ return k;
+ int s = size[0];
+ k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
+ if (k != 0)
+ return k;
+ size[0] += s+1;
+ return 0;
+ }
+
+ final public static int compare3 ( byte[] x, int xs, int xl, byte[] y, int ys, int yl, int[] size ) {
+ int k = MRContainer.compare(x,xs,xl,y,ys,yl,size);
+ if (k != 0)
+ return k;
+ int s = size[0];
+ k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
+ if (k != 0)
+ return k;
+ s += size[0];
+ k = MRContainer.compare(x,xs+s,xl-s,y,ys+s,yl-s,size);
+ if (k != 0)
+ return k;
+ size[0] += s+1;
+ return 0;
+ }
+
+ public boolean equals ( Object x ) {
+ if (!(x instanceof Tuple))
+ return false;
+ Tuple xt = (Tuple) x;
+ if (xt.tuple.length != tuple.length)
+ return false;
+ for ( short i = 0; i < tuple.length; i++ )
+ if (!xt.get(i).equals(get(i)))
+ return false;
+ return true;
+ }
+
+ public int hashCode () {
+ int h = 127;
+ for ( short i = 1; i < tuple.length; i++ )
+ h ^= get(i).hashCode();
+ return Math.abs(h);
+ }
+
+ public String toString () {
+ if (size() == 0)
+ return "()";
+ String s = "("+get((short)0);
+ for ( short i = 1; i < tuple.length; i++ )
+ s += ","+get(i);
+ return s+")";
+ }
+}
[03/26] MRQL-32: Refactoring directory structure for Eclipse
Posted by fe...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/XPathParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/core/XPathParser.java b/src/main/java/core/XPathParser.java
deleted file mode 100644
index faded33..0000000
--- a/src/main/java/core/XPathParser.java
+++ /dev/null
@@ -1,692 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import org.apache.mrql.gen.*;
-import java.util.Stack;
-import org.xml.sax.*;
-import org.xml.sax.helpers.DefaultHandler;
-import org.xml.sax.helpers.AttributesImpl;
-import java.io.PrintStream;
-import java.nio.ByteBuffer;
-
-
-/** Compiles XPath queries to SAX pipelines */
-final public class XPathParser {
- public MRDataHandler dataConstructor;
- public XPathHandler handler;
- final static PrintStream out = System.out;
- static int cached_events = 0;
-
- /** a SAX handler to form XPath pipelines */
- abstract class XPathHandler extends DefaultHandler {
- XPathHandler next; // the next handler in the pipeline
-
- public XPathHandler ( XPathHandler next ) {
- this.next = next;
- }
-
- public void startDocument () throws SAXException {
- next.startDocument();
- }
-
- public void endDocument () throws SAXException {
- next.endDocument();
- }
-
- abstract public void startElement ( String uri, String name, String tag, Attributes atts ) throws SAXException;
-
- abstract public void endElement ( String uri, String name, String tag ) throws SAXException;
-
- abstract public void characters ( char text[], int start, int length ) throws SAXException;
-
- /** start a new predicate */
- public void startPredicate ( int pred ) {
- next.startPredicate(pred);
- }
-
- /** the end of a predicate */
- public void endPredicate ( int pred ) {
- next.endPredicate(pred);
- }
-
- public void releasePredicate ( int pred ) { // set the predicate outcome to true
- next.releasePredicate(pred);
- }
- }
-
- /** The end of the pipeline: Print the SAX stream to the output */
- final class Print extends XPathHandler {
-
- public Print () {
- super(null);
- }
-
- public void startDocument () {}
-
- public void endDocument () { out.println(); }
-
- public void startElement ( String uri, String name, String tag, Attributes atts ) {
- out.append("<").append(tag);
- if (atts != null)
- for (int i = 0; i < atts.getLength(); i++)
- out.append(" ").append(atts.getQName(i))
- .append("=\"").append(atts.getValue(i)).append("\"");
- out.append(">");
- }
-
- public void endElement ( String uri, String name, String tag ) {
- out.append("</").append(tag).append(">");
- }
-
- public void characters ( char text[], int start, int length ) {
- for (int i = 0; i < length; i++)
- out.append(text[start+i]);
- }
-
- public String toString () { return "print()"; }
- }
-
- /** A growable buffer for storing events as byte sequences */
- final class Cache {
- final static int buffer_size_increment = 1000;
- public byte[] wrapped_buffer;
- public ByteBuffer byteBuffer;
-
- public Cache () {
- wrapped_buffer = new byte[buffer_size_increment];
- byteBuffer = ByteBuffer.wrap(wrapped_buffer);
- }
-
- private void grow ( int len ) {
- len /= java.lang.Byte.SIZE;
- while (len+byteBuffer.position() > byteBuffer.limit()) {
- int pos = byteBuffer.position();
- byte[] nb = new byte[wrapped_buffer.length+buffer_size_increment];
- for (int i = 0; i < wrapped_buffer.length; i++)
- nb[i] = wrapped_buffer[i];
- wrapped_buffer = nb;
- byteBuffer = ByteBuffer.wrap(nb);
- byteBuffer.position(pos);
- }
- }
-
- public void putByte ( byte n ) {
- grow(8);
- byteBuffer.put(n);
- }
-
- public void putShort ( short n ) {
- grow(Short.SIZE);
- byteBuffer.putShort(n);
- }
-
- public void putInt ( int n ) {
- grow(Integer.SIZE);
- byteBuffer.putInt(n);
- }
-
- public void putChars ( char text[], int start, int len ) {
- grow(len*Character.SIZE+Integer.SIZE);
- byteBuffer.putInt(len);
- for (int i = 0; i < len; i++)
- byteBuffer.putChar(text[start+i]);
- }
-
- public void putString ( String s ) {
- grow(s.length()*Character.SIZE+Short.SIZE);
- int len = s.length();
- byteBuffer.putShort((short) len);
- for (int i = 0; i < len; i++)
- byteBuffer.putChar(s.charAt(i));
- }
-
- public byte getByte () { return byteBuffer.get(); }
-
- public short getShort () { return byteBuffer.getShort(); }
-
- public int getInt () { return byteBuffer.getInt(); }
-
- public char[] getChars () {
- int len = byteBuffer.getInt();
- char[] buf = new char[len];
- for (int i = 0; i < len; i++)
- buf[i] = byteBuffer.getChar();
- return buf;
- }
-
- public String getString () {
- int len = byteBuffer.getShort();
- char[] buf = new char[len];
- for (int i = 0; i < len; i++)
- buf[i] = byteBuffer.getChar();
- return new String(buf);
- }
-
- public void cacheStartElement ( String uri, String name, String tag, Attributes atts ) {
- cached_events++;
- putByte((byte)0);
- putString(uri);
- putString(name);
- putString(tag);
- if (atts != null) {
- putShort((short) atts.getLength());
- for (int i = 0; i < atts.getLength(); i++) {
- putString(atts.getQName(i));
- putString(atts.getValue(i));
- }
- } else putShort((short) 0);
- }
-
- public void cacheEndElement ( String uri, String name, String tag ) {
- cached_events++;
- putByte((byte)1);
- putString(uri);
- putString(name);
- putString(tag);
- }
-
- public void cacheCharacters ( char text[], int start, int length ) {
- cached_events++;
- putByte((byte)2);
- putChars(text,start,length);
- }
-
- public void print () {
- System.out.println(byteBuffer);
- dump(new Print());
- }
-
- public void append ( Cache cache ) {
- grow(cache.byteBuffer.position());
- byte[] b = cache.byteBuffer.array();
- byteBuffer.put(b,0,cache.byteBuffer.position());
- cache.byteBuffer.clear();
- }
-
- /** regenerate the stream from buffer */
- public void dump ( XPathHandler next ) {
- int last = byteBuffer.position();
- byteBuffer.position(0);
- while (byteBuffer.position() < last)
- try {
- switch (getByte()) {
- case 0:
- String uri = getString();
- String name = getString();
- String tag = getString();
- AttributesImpl atts = new AttributesImpl();
- int len = getShort();
- for (int i = 0; i < len; i++)
- atts.addAttribute("","",getString(),"",getString());
- next.startElement(uri,name,tag,atts);
- break;
- case 1:
- next.endElement(getString(),getString(),getString());
- break;
- case 2:
- char[] text = getChars();
- next.characters(text,0,text.length);
- }
- } catch (SAXException e) {
- throw new Error(e);
- };
- byteBuffer.clear();
- }
- }
-
- /** Remove the start/end/releasePredicate events by storing some events in a buffer, when necessary */
- final class Materialize extends XPathHandler {
- final static int max_num_of_nested_predicates = 100;
- final public Cache cache; // nested suspended events from predicates whose outcome is unknown
- final int[] ids; // the ids of the predicates with suspended output
- final int[] positions; // position of predicate events in the buffer
- final boolean[] released; // true if the associated predicate is true
- int top; // top of the stacks
-
- public Materialize ( XPathHandler next ) {
- super(next);
- cache = new Cache();
- ids = new int[max_num_of_nested_predicates];
- positions = new int[max_num_of_nested_predicates];
- released = new boolean[max_num_of_nested_predicates];
- top = 0;
- }
-
- public void startElement ( String uri, String name, String tag, Attributes atts ) throws SAXException {
- if (top > 0)
- cache.cacheStartElement(uri,name,tag,atts);
- else next.startElement(uri,name,tag,atts);
- }
-
- public void endElement ( String uri, String name, String tag ) throws SAXException {
- if (top > 0)
- cache.cacheEndElement(uri,name,tag);
- else next.endElement(uri,name,tag);
- }
-
- public void characters ( char text[], int start, int length ) throws SAXException {
- if (top > 0)
- cache.cacheCharacters(text,start,length);
- else next.characters(text,start,length);
- }
-
- public void startPredicate ( int pred ) {
- if (top >= ids.length)
- throw new Error("too many nested predicates");
- positions[top] = cache.byteBuffer.position();
- ids[top] = pred;
- released[top++] = false;
- }
-
- public void endPredicate ( int pred ) {
- if (top > 0 && ids[top-1] == pred)
- cache.byteBuffer.position(positions[--top]).mark().reset();
- }
-
- public void releasePredicate ( int pred ) {
- boolean flush = true;
- for (int i = 0; i < top; i++)
- if (ids[i] == pred)
- released[i] = true;
- else flush &= released[i];
- if (top > 0 && flush) {
- cache.dump(next);
- top = 0;
- }
- }
-
- public String toString () { return "materialize("+next+")"; }
- }
-
- /** return the children of the current nodes that have the given tagname */
- final class Child extends XPathHandler {
- final String tagname; // the tagname of the child
- boolean keep; // are we keeping or skipping events?
- short level; // the depth level of the current element
-
- public Child ( String tagname, XPathHandler next ) {
- super(next);
- this.tagname = tagname;
- keep = false;
- level = 0;
- }
-
- public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
- if (level++ == 1)
- keep = tagname.equals("*") || tagname.equals(qn);
- if (keep)
- next.startElement(nm,ln,qn,a);
- }
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {
- if (keep)
- next.endElement(nm,ln,qn);
- if (--level == 1)
- keep = false;
- }
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- if (keep)
- next.characters(text,start,length);
- }
-
- public String toString () { return "child("+tagname+","+next+")"; }
- }
-
- /** return the attribute value of the current nodes that have the given attribute name */
- final class Attribute extends XPathHandler {
- final String attributename;
- short level; // the depth level of the current element
-
- public Attribute ( String attribute_name, XPathHandler next ) {
- super(next);
- attributename = attribute_name;
- level = 0;
- }
-
- public void startElement ( String nm, String ln, String qn, Attributes as ) throws SAXException {
- if (level++ == 0)
- for ( int i = 0; i < as.getLength(); i++ )
- if (attributename.equals("*") || attributename.equals(as.getQName(i))) {
- char[] s = as.getValue(i).toCharArray();
- next.characters(s,0,s.length);
- }
- }
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {
- --level;
- }
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- }
-
- public String toString () { return "attribute("+attributename+","+next+")"; }
- }
-
- /** Return the descendants of the current nodes that have the given tagname.
- * To handle nested elements with the same tagname, use Descendant
- */
- final class SimpleDescendant extends XPathHandler {
- final String tagname; // the tagname of the descendant
- boolean keep; // are we keeping or skipping events?
-
- public SimpleDescendant ( String tagname, XPathHandler next ) {
- super(next);
- this.tagname = tagname;
- keep = false;
- }
-
- public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
- if (!keep)
- keep = tagname.equals(qn);
- if (keep)
- next.startElement(nm,ln,qn,a);
- }
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {
- if (keep) {
- next.endElement(nm,ln,qn);
- keep = !tagname.equals(qn);
- }
- }
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- if (keep)
- next.characters(text,start,length);
- }
-
- public String toString () { return "simple_descendant("+tagname+","+next+")"; }
- }
-
- /** As efficient as SimpleDescendant when there are no nested elements with the same tagname.
- * It caches only the inner nested subelements with the same tagname.
- */
- final class Descendant extends XPathHandler {
- final static int max_nested_level = 100;
- final String tagname; // the tagname of the descendant
- int level; // # of nested elements with the same tagname
- final Cache[] cache; // cache[i] caches elements of level i-1
-
- public Descendant ( String tagname, XPathHandler next ) {
- super(next);
- this.tagname = tagname;
- cache = new Cache[max_nested_level]; // to be created lazily
- level = 0;
- }
-
- public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
- if (tagname.equals(qn)) {
- if (level > 0 && cache[level-1] == null)
- cache[level-1] = new Cache();
- level++;
- };
- for (int i = 1; i < level; i++)
- cache[i-1].cacheStartElement(nm,ln,qn,a);
- if (level > 0)
- next.startElement(nm,ln,qn,a);
- }
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {
- if (level > 0)
- next.endElement(nm,ln,qn);
- for (int i = 1; i < level; i++)
- cache[i-1].cacheEndElement(nm,ln,qn);
- if (tagname.equals(qn)) {
- level--;
- if (level == 0 && cache[0] != null)
- cache[0].dump(next);
- else if (level > 0 && cache[level] != null)
- cache[level-1].append(cache[level]);
- }
- }
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- for (int i = 1; i < level; i++)
- cache[i-1].cacheCharacters(text,start,length);
- if (level > 0)
- next.characters(text,start,length);
- }
-
- public String toString () { return "descendant("+tagname+","+next+")"; }
- }
-
- /** propagates all input signals to both next and condition streams but wraps each
- * top-level element in the next stream with start/end Condition signals
- */
- final class Predicate extends XPathHandler {
- int level;
- final XPathHandler condition; // false, if stream is empty
- final int predicate_id; // the id of the predicate
-
- public Predicate ( int predicate_id, XPathHandler condition, XPathHandler next ) {
- super(next);
- this.condition = condition;
- this.predicate_id = predicate_id;
- level = 0;
- }
-
- public void startPredicate ( int pred ) {
- next.startPredicate(pred);
- condition.startPredicate(pred);
- }
-
- public void endPredicate ( int pred ) {
- next.endPredicate(pred);
- condition.endPredicate(pred);
- }
-
- public void releasePredicate ( int pred ) {
- next.releasePredicate(pred);
- condition.releasePredicate(pred);
- }
-
- public void startDocument () throws SAXException {
- next.startDocument();
- condition.startDocument();
- }
-
- public void endDocument () throws SAXException {
- next.endDocument();
- condition.endDocument();
- }
-
- public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {
- if (level++ == 0)
- next.startPredicate(predicate_id);
- next.startElement(nm,ln,qn,a);
- condition.startElement(nm,ln,qn,a);
- }
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {
- next.endElement(nm,ln,qn);
- condition.endElement(nm,ln,qn);
- if (--level == 0)
- next.endPredicate(predicate_id);
- }
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- next.characters(text,start,length);
- condition.characters(text,start,length);
- }
-
- public String toString () { return "predicate("+predicate_id+","+condition+","+next+")"; }
- }
-
- /** generate a releasePredicate signal if the content of the input node is equal to text */
- final class Equals extends XPathHandler {
- final static int max_depth_of_nested_predicates = 100;
- final String value; // the value to be tested for equality
- final int predicate_id; // the id of the predicate
- final int[] preds;
- int top;
- boolean suspended;
-
- public Equals ( int predicate_id, String value, XPathHandler next ) {
- super(next);
- this.value = value;
- this.predicate_id = predicate_id;
- preds = new int[max_depth_of_nested_predicates];
- top = 0;
- suspended = false;
- }
-
- private boolean compare ( char[] text, int start, int length, String value ) {
- if (length != value.length())
- return false;
- for (int i = 0; i < length; i++)
- if (text[i+start] != value.charAt(i))
- return false;
- return true;
- }
-
- public void startPredicate ( int pred ) {
- preds[top++] = pred;
- }
-
- public void endPredicate ( int pred ) {
- suspended = false;
- for (int i = 0; i < top; i++)
- if (preds[i] == pred) {
- preds[i] = preds[--top];
- return;
- }
- }
-
- public void releasePredicate ( int pred ) {
- if (top == 1 && suspended)
- next.releasePredicate(predicate_id);
- endPredicate(pred);
- }
-
- public void startDocument () throws SAXException {}
-
- public void endDocument () throws SAXException {}
-
- public void startElement ( String nm, String ln, String qn, Attributes a ) throws SAXException {}
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {}
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- if (compare(text,start,length,value))
- if (top == 0)
- next.releasePredicate(predicate_id);
- else suspended = true;
- }
-
- public String toString () { return "equals("+predicate_id+","+value+","+next+")"; }
- }
-
- /** Converts the SAX data stream to MRData */
- final class MRDataHandler extends XPathHandler {
- Stack<Union> stack = new Stack<Union>();
- Bag children;
-
- public MRDataHandler () throws Exception {
- super(null);
- stack.clear();
- Tuple t = new Tuple(3);
- children = new Bag();
- t.set(2,children);
- stack.push(new Union((byte)0,t));
- }
-
- public void start () throws Exception {
- stack.clear();
- Tuple t = new Tuple(3);
- children = new Bag();
- t.set(2,children);
- stack.push(new Union((byte)0,t));
- }
-
- public Bag value () {
- if (stack.size() != 1)
- return null;
- else return ((Bag)((Tuple)stack.peek().value()).get(2));
- }
-
- public void startDocument () throws SAXException {}
-
- public void endDocument () throws SAXException {}
-
- public void startElement ( String nm, String ln, String qn, Attributes as ) throws SAXException {
- children = new Bag();
- Bag attributes = new Bag();
- for ( int i = 0; i < as.getLength(); i++ )
- attributes.add(new Tuple(new MR_string(as.getQName(i)),new MR_string(as.getValue(i))));
- Tuple t = new Tuple(3);
- t.set(0,new MR_string(qn));
- t.set(1,attributes);
- t.set(2,children);
- stack.push(new Union((byte)0,t));
- }
-
- public void endElement ( String nm, String ln, String qn ) throws SAXException {
- if (stack.empty())
- throw new SAXException("Ill-formed XML elements: "+qn);
- Union v = stack.pop();
- if (!((MR_string)((Tuple)v.value()).get(0)).get().equals(qn))
- throw new SAXException("Unmatched tags in XML element: "+qn);
- children = (Bag)((Tuple)stack.peek().value()).get(2);
- children.add(v);
- }
-
- public void characters ( char[] text, int start, int length ) throws SAXException {
- String s = new String(text,start,length);
- if (s.startsWith("{{") && s.endsWith("}}"))
- children.add(new MR_variable(Integer.parseInt(s.substring(2,s.length()-2))));
- else children.add(new Union((byte)1,new MR_string(s)));
- }
-
- public String toString () { return "MRDataHandler()"; }
- }
-
- public XPathParser ( Tree xpath ) throws Exception {
- dataConstructor = new MRDataHandler();
- handler = compile_xpath(xpath,new Materialize(dataConstructor),0);
- }
-
- public XPathHandler compile_xpath ( Tree xpath, XPathHandler next, int cn ) throws Exception {
- if (xpath.is_variable() && xpath.toString().equals("dot"))
- return next;
- if (!xpath.is_node())
- throw new Error("Unrecognized xpath query: "+xpath);
- Node n = (Node) xpath;
- if (n.name().equals("child")) {
- String tag = n.children().head().toString();
- XPathHandler c = (tag.charAt(0) == '@')
- ? new Attribute(tag.substring(1),next)
- : new Child(tag,next);
- return compile_xpath(n.children().tail().head(),c,cn);
- } else if (n.name().equals("descendant")) {
- XPathHandler c = new Descendant(n.children().head().toString(),next);
- return compile_xpath(n.children().tail().head(),c,cn);
- } else if (n.name().equals("eq")) {
- Tree value = n.children().tail().head();
- XPathHandler c = new Equals(cn,(value.is_string())
- ? ((StringLeaf)value).value()
- : value.toString(),
- next);
- return compile_xpath(n.children().head(),c,cn);
- } else if (n.name().equals("predicate")) {
- XPathHandler c = new Predicate(cn+1,compile_xpath(n.children().head(),next,cn+1),next);
- return compile_xpath(n.children().tail().head(),c,cn);
- };
- throw new Error("Unrecognized xpath query: "+xpath);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/mrql.cgen
----------------------------------------------------------------------
diff --git a/src/main/java/core/mrql.cgen b/src/main/java/core/mrql.cgen
deleted file mode 100644
index 9b8dbe4..0000000
--- a/src/main/java/core/mrql.cgen
+++ /dev/null
@@ -1,365 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java_cup.runtime.*;
-import org.apache.mrql.gen.*;
-
-parser code {:
-
- static int[] tokens = {
- sym.error, sym.IF, sym.THEN, sym.ELSE, sym.SELECT, sym.FROM, sym.HAVING,
- sym.LB, sym.RB, sym.LP, sym.RP, sym.LSB, sym.RSB,
- sym.PLUS, sym.MINUS, sym.TIMES, sym.DIV, sym.MOD, sym.EQ,
- sym.NEQ, sym.LT, sym.LEQ, sym.GT, sym.GEQ, sym.SHARP, sym.AS,
- sym.AND, sym.OR, sym.NOT, sym.UNION,
- sym.INTERSECT, sym.EXCEPT, sym.EXISTS, sym.IN, sym.DOT, sym.COLON, sym.COMMA,
- sym.SEMI, sym.ASSIGN, sym.WHERE, sym.ORDER, sym.GROUP, sym.BY, sym.ASCENDING,
- sym.DESCENDING, sym.FUNCTION, sym.DISTINCT, sym.BSLASH, sym.SOME, sym.ALL,
- sym.SOME, sym.ALL, sym.GTR, sym.SEP, sym.STORE, sym.DUMP, sym.TYPE, sym.DATA, sym.REPEAT,
- sym.STEP, sym.LIMIT, sym.LET, sym.ATSYM, sym.EXCLAMATION,
- sym.Variable, sym.Integer, sym.Double, sym.String, sym.Decimal,
- sym.START_TEMPLATE, sym.END_TEMPLATE, sym.TEXT
- };
-
- static String[] token_names = {
- "error", "if", "then", "else", "select", "from", "having",
- "[", "]", "(", ")", "{", "}",
- "+", "-", "*", "/", "mod", "=",
- "<>", "<", "<=", ">", ">=", "#", "as",
- "and", "or", "not", "union",
- "intersect", "except", "exists", "in", ".", ":", ",",
- ";", ":=", "where", "order", "group", "by", "ascending",
- "descending", "function", "distinct", "\\", "some", "all",
- "some", "all", ">", "|", "store", "dump", "type", "data", "repeat",
- "step", "limit", "let", "@", "!",
- "Variable", "Integer", "Double", "String", "Decimal",
- "[|", "|]", "Text"
- };
-
- public static String print ( Symbol s ) {
- for (int i=0; i<tokens.length; i++)
- if (tokens[i]==s.sym)
- { String res = token_names[i] + " ";
- if (s.value==null)
- return res;
- else if (s.value instanceof Integer)
- return res + ((Integer) s.value).intValue();
- else if (s.value instanceof Float)
- return res + ((Float) s.value).floatValue();
- else if (s.value instanceof String)
- return res + (String) s.value;
- }
- return "?";
- }
-
- static {
- Tree.parsed = true;
- }
-
- public int line_pos () {
- return ((MRQLLex)getScanner()).line_pos();
- }
-
- public int char_pos () {
- return ((MRQLLex)getScanner()).char_pos();
- }
-
- public void syntax_error ( Symbol token ) {
- System.err.println("*** Syntax Error: " + print(token) + " (line: " + line_pos() + ", position: " + char_pos() + ")");
- if (Config.testing)
- throw new Error("Syntax Error");
- }
-
-:};
-
-/* Terminals (tokens returned by the scanner). */
-terminal IF, THEN, ELSE, SELECT, FROM, HAVING, LB, RB, LP, RP, LSB, RSB, LDOT, SHARP,
- PLUS, MINUS, TIMES, DIV, MOD, EQ, NEQ, LT, LEQ, GT, GEQ, AND, OR, NOT, AS,
- UNION, INTERSECT, EXCEPT, EXISTS, IN, COMMA, DOT, COLON, ASSIGN, SEMI, WHERE,
- ORDER, GROUP, BY, ASCENDING, DESCENDING, UMINUS, FUNCTION, DISTINCT, BSLASH,
- SOME, ALL, GTR, SEP, STORE, TYPE, DATA, CASE, ATSYM, XPATH, REPEAT, STEP, LIMIT,
- LET, IMPORT, PARSER, AGGREGATION, INCLUDE, EXCLAMATION, MACRO, DUMP;
-
-terminal String Variable;
-terminal Long Integer;
-terminal Double Double;
-terminal String String;
-terminal Double Decimal;
-terminal String START_TEMPLATE;
-terminal String END_TEMPLATE;
-terminal String TEXT;
-
-non terminal prog;
-non terminal Tree item, expr, var, const, mrql, pattern, opt_where, opt_orderby,
- mode, opt_groupby, opt_having, opt_distinct, type, xpath, xname,
- xpred, opt_limit, unit;
-non terminal Trees expl, name_binds, pat_list, pat_binds,
- binds, order_binds, groupby_binds, typel, type_binds,
- data_binds, cases, fnl, template, template_pat, fnc_params, var_list;
-
-precedence nonassoc LDOT, ASSIGN, LIMIT;
-precedence nonassoc ELSE, COLON;
-precedence nonassoc ORDER, GROUP, HAVING, WHERE;
-precedence left INTERSECT, UNION, EXCEPT, IN;
-precedence nonassoc ASCENDING, DESCENDING;
-precedence nonassoc COMMA, LP;
-precedence right OR;
-precedence right AND;
-precedence nonassoc NOT;
-precedence nonassoc EQ, LT, GT, LEQ, GEQ, NEQ;
-precedence left PLUS, MINUS;
-precedence left TIMES, DIV, MOD;
-precedence nonassoc DOT, SHARP, LB, AS;
-precedence nonassoc UMINUS;
-
-start with prog;
-
-prog ::= item:i SEMI {: Translator.top_level(i); :}
- | prog item:i SEMI {: Translator.top_level(i); :}
- ;
-item ::= expr:e {: RESULT = #<expression(`e)>; :}
- | var:v EQ expr:e {: RESULT = #<assign(`v,`e)>; :}
- | STORE var:v ASSIGN expr:e {: RESULT = #<store(`v,`e)>; :}
- | STORE String:s FROM expr:e {: RESULT = #<dump(`(new StringLeaf(s)),`e)>; :}
- | DUMP String:s FROM expr:e {: RESULT = #<dump_text(`(new StringLeaf(s)),`e)>; :}
- | TYPE var:v EQ type:t {: RESULT = #<typedef(`v,`t)>; :}
- | DATA var:v EQ data_binds:nl {: RESULT = #<datadef(`v,union(...nl))>; :}
- | FUNCTION var:f LP
- fnc_params:p RP COLON type:tp
- LSB expr:e RSB {: RESULT = #<functiondef(`f,params(...p),`tp,`e)>; :}
- | FUNCTION var:f LP RP COLON type:tp
- LSB expr:e RSB {: RESULT = #<functiondef(`f,params(),`tp,`e)>; :}
- | MACRO var:f LP var_list:vl RP
- LSB expr:e RSB {: RESULT = #<macrodef(`f,params(...vl),`e)>; :}
- | IMPORT fnl:l FROM String:c {: RESULT = #<import(`c,...l)>; :}
- | IMPORT String:c {: RESULT = #<import(`c)>; :}
- | PARSER var:c EQ String:p {: RESULT = #<parser(`c,`p)>; :}
- | INCLUDE String:c {: RESULT = #<include(`c)>; :}
- | AGGREGATION var:n LP expr:a COMMA expr:z unit:u RP COLON type:tp
- {: RESULT = #<aggregation(`n,`tp,`a,`z,`u)>; :}
- | error {: :}
- ;
-unit ::= COMMA expr:u {: RESULT = u; :}
- | {: RESULT = #<lambda(x,x)>; :}
- ;
-fnl ::= Variable:v {: RESULT = #[`v]; :}
- | String:v {: RESULT = #[`v]; :}
- | Variable:v COMMA fnl:l {: RESULT = #[`v,...l]; :}
- | String:v COMMA fnl:l {: RESULT = #[`v,...l]; :}
- ;
-fnc_params ::= fnc_params:nl
- COMMA var:v COLON type:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
- | var:v COLON type:e {: RESULT = #[bind(`v,`e)]; :}
- ;
-var ::= Variable:v {: RESULT = #<`v>; :}
- ;
-var_list ::= var:v {: RESULT = #[`v]; :}
- | var_list:el COMMA var:v {: RESULT = el.append(v); :}
- ;
-const ::= String:s {: RESULT = new StringLeaf(s); :}
- | Integer:n {: RESULT = new LongLeaf(n.longValue()); :}
- | Double:n {: RESULT = new DoubleLeaf((double)n.doubleValue()); :}
- | Decimal:n {: RESULT = new DoubleLeaf((double)n.doubleValue()); :}
- ;
-expr ::= mrql:e {: RESULT = e; :}
- | var:v {: RESULT = v; :}
- | IF expr:p THEN expr:e1 ELSE expr:e2 {: RESULT = #<if(`p,`e1,`e2)>; :}
- | expr:e1 PLUS expr:e2 {: RESULT = #<call(plus,`e1,`e2)>; :}
- | expr:e1 MINUS expr:e2 {: RESULT = #<call(minus,`e1,`e2)>; :}
- | expr:e1 TIMES expr:e2 {: RESULT = #<call(times,`e1,`e2)>; :}
- | expr:e1 DIV expr:e2 {: RESULT = #<call(div,`e1,`e2)>; :}
- | expr:e1 MOD expr:e2 {: RESULT = #<call(mod,`e1,`e2)>; :}
- | expr:e1 EQ expr:e2 {: RESULT = #<call(eq,`e1,`e2)>; :}
- | expr:e1 NEQ expr:e2 {: RESULT = #<call(neq,`e1,`e2)>; :}
- | expr:e1 LT expr:e2 {: RESULT = #<call(lt,`e1,`e2)>; :}
- | expr:e1 LEQ expr:e2 {: RESULT = #<call(leq,`e1,`e2)>; :}
- | expr:e1 GT expr:e2 {: RESULT = #<call(gt,`e1,`e2)>; :}
- | expr:e1 GEQ expr:e2 {: RESULT = #<call(geq,`e1,`e2)>; :}
- | expr:e1 AND expr:e2 {: RESULT = #<call(and,`e1,`e2)>; :}
- | expr:e1 OR expr:e2 {: RESULT = #<call(or,`e1,`e2)>; :}
- | expr:e1 UNION expr:e2 {: RESULT = #<call(plus,`e1,`e2)>; :}
- | expr:e1 INTERSECT expr:e2 {: RESULT = #<intersect(`e1,`e2)>; :}
- | expr:e1 EXCEPT expr:e2 {: RESULT = #<except(`e1,`e2)>; :}
- | expr:e DOT var:v {: RESULT = #<project(`e,`v)>; :}
- | expr:e DOT TIMES {: RESULT = #<call(XMLchildren,`(new StringLeaf("*")),`e)>; :}
- | expr:e DOT ATSYM Variable:v {: RESULT = #<call(XMLattribute,`(new StringLeaf(v)),`e)>; :}
- | expr:e DOT ATSYM TIMES {: RESULT = #<call(XMLattributes,`(new StringLeaf("*")),`e)>; :}
- | expr:e SHARP Integer:n {: RESULT = #<nth(`e,`(new LongLeaf(n.longValue())))>; :}
- | expr:e LB expr:i RB {: RESULT = #<index(`e,`i)>; :}
- | expr:e LB expr:i COLON expr:j RB {: RESULT = #<range(`e,`i,`j)>; :}
- | var:v LP RP {: RESULT = #<call(`v)>; :}
- | var:v LP expl:el RP {: RESULT = #<call(`v,...el)>; :}
- | LP RP {: RESULT = #<tuple()>; :}
- | LP expl:el RP {: RESULT = (el.length()==1)?el.head():#<tuple(...el)>; :}
- | LSB RSB {: RESULT = #<bag()>; :}
- | LSB expl:el RSB {: RESULT = #<bag(...el)>; :}
- | LB RB {: RESULT = #<list()>; :}
- | LB expl:el RB {: RESULT = #<list(...el)>; :}
- | LT {: MRQLLex.record_begin(); :}
- name_binds:nl GTR {: MRQLLex.record_end(); RESULT = #<record(...nl)>; :}
- | PLUS expr:e {: RESULT = e; :} %prec UMINUS
- | MINUS expr:e {: RESULT = #<call(minus,`e)>; :} %prec UMINUS
- | NOT expr:e {: RESULT = #<call(not,`e)>; :}
- | EXISTS expr:e {: RESULT = #<call(exists,`e)>; :}
- | CASE expr:e LSB cases:cs RSB {: RESULT = #<case(`e,...cs)>; :}
- | BSLASH LP fnc_params:p RP
- COLON type:tp DOT expr:e {: RESULT = #<function(tuple(...p),`tp,`e)>; :} %prec LDOT
- | expr:e AS type:tp {: RESULT = #<typed(`e,`tp)>; :}
- | LET pattern:p EQ expr:e IN expr:b {: RESULT = #<let_bind(`p,`e,`b)>; :}
- | expr:i DOT DOT expr:j {: RESULT = #<range(`i,`j)>; :}
- | expr:i DOT DOT DOT expr:j {: RESULT = #<gen(`i,`j,-1)>; :}
- | XPATH LP xpath:x RP {: RESULT = #<xpath(`x)>; :}
- | TYPE LP type:tp RP {: RESULT = #<type(`tp)>; :}
- | REPEAT Variable:v EQ expr:s STEP expr:b
- {: RESULT = #<repeat(lambda(`v,`b),`s)>; :}
- | REPEAT Variable:v EQ expr:s STEP expr:b LIMIT expr:n
- {: RESULT = #<repeat(lambda(`v,`b),`s,`n)>; :}
- | REPEAT LP var_list:vl RP EQ expr:s STEP expr:b LIMIT expr:n
- {: RESULT = #<loop(lambda(tuple(...vl),`b),`s,`n)>; :}
- | START_TEMPLATE:s template:t END_TEMPLATE:e
- {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),...t,text(`e))>); :}
- | START_TEMPLATE:s END_TEMPLATE:e {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),text(`e))>); :}
- | const:c {: RESULT = c; :}
- ;
-template ::= template:t TEXT:s {: RESULT = t.append(#<text(`s)>); :}
- | template:t expr:e {: RESULT = t.append(e); :}
- | TEXT:s {: RESULT = #[text(`s)]; :}
- | expr:e {: RESULT = #[`e]; :}
- ;
-expl ::= expr:e {: RESULT = #[`e]; :}
- | expl:el COMMA expr:e {: RESULT = el.append(e); :}
- ;
-name_binds ::= name_binds:nl
- COMMA var:v COLON expr:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
- | var:v COLON expr:e {: RESULT = #[bind(`v,`e)]; :}
- ;
-cases ::= cases:cs SEP pattern:p COLON expr:e {: RESULT = cs.append(#<case(`p,`e)>); :}
- | pattern:p COLON expr:e {: RESULT = #[case(`p,`e)]; :}
- ;
-pattern ::= var:v {: RESULT = v; :}
- | TIMES {: RESULT = #<any>; :}
- | const:c {: RESULT = c; :}
- | LP RP {: RESULT = #<tuple()>; :}
- | LP pat_list:el RP {: RESULT = (el.length()==1)?el.head():#<tuple(...el)>; :}
- | LB RB {: RESULT = #<list()>; :}
- | LB expl:el RB {: RESULT = #<list(...el)>; :}
- | pattern:p AS type:t {: RESULT = #<typed(`p,`t)>; :}
- | Variable:c LP pat_list:pl RP {: RESULT = #<call(`c,...pl)>; :}
- | LT {: MRQLLex.record_begin(); :}
- pat_binds:nl GTR {: MRQLLex.record_end(); RESULT = #<record(...nl)>; :}
- | START_TEMPLATE:s template_pat:t END_TEMPLATE:e
- {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),...t,text(`e))>); :}
- | START_TEMPLATE:s END_TEMPLATE:e {: RESULT = Translator.template(#<template(`(s.substring(1,s.length()-1)),text(`e))>); :}
- ;
-template_pat ::= template_pat:t TEXT:s {: RESULT = t.append(#<text(`s)>); :}
- | template_pat:t pattern:e {: RESULT = t.append(e); :}
- | TEXT:s {: RESULT = #[text(`s)]; :}
- | pattern:e {: RESULT = #[`e]; :}
- ;
-pat_list ::= pattern:e {: RESULT = #[`e]; :}
- | pat_list:el COMMA pattern:e {: RESULT = el.append(e); :}
- ;
-pat_binds ::= pat_binds:nl
- COMMA var:v COLON pattern:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
- | var:v COLON pattern:e {: RESULT = #[bind(`v,`e)]; :}
- ;
-mrql ::= SELECT opt_distinct:d expr:e
- FROM binds:bl
- opt_where:w
- opt_groupby:g
- opt_orderby:o {: RESULT = #<select(`d,`e,from(...bl),`w,`g,`o)>; :}
- | SOME binds:bl COLON expr:e {: RESULT = #<call(exists,select(none,true,from(...bl),where(`e),groupby(),orderby()))>; :}
- | ALL binds:bl COLON expr:e {: RESULT = #<call(all,select(none,`e,from(...bl),where(true),groupby(),orderby()))>; :}
- | ALL LP mrql:e RP {: RESULT = #<call(all,`e)>; :}
- | SOME LP mrql:e RP {: RESULT = #<call(some,`e)>; :}
- ;
-opt_distinct ::= DISTINCT {: RESULT = #<distinct>; :}
- | {: RESULT = #<none>; :}
- ;
-binds ::= binds:bl COMMA pattern:p IN expr:e {: RESULT = bl.append(#<bind(`p,`e)>); :}
- | binds:bl COMMA pattern:p EQ expr:e {: RESULT = bl.append(#<bind(`p,bag(`e))>); :}
- | pattern:p IN expr:e {: RESULT = #[bind(`p,`e)]; :}
- | pattern:p EQ expr:e {: RESULT = #[bind(`p,bag(`e))]; :}
- ;
-opt_where ::= WHERE expr:e {: RESULT = #<where(`e)>; :}
- | {: RESULT = #<where(true)>; :}
- ;
-opt_orderby ::= ORDER BY order_binds:ol opt_limit:l {: RESULT = #<orderby(`l,...ol)>; :}
- | {: RESULT = #<orderby()>; :}
- ;
-order_binds ::= expr:e mode:m {: RESULT = m.equals(#<asc>) ? #[`e] : #[call(inv,`e)]; :}
- | order_binds:ol COMMA expr:e mode:m {: RESULT = ol.append(m.equals(#<asc>) ? #<`e> : #<call(inv,`e)>); :}
- ;
-mode ::= ASCENDING {: RESULT = #<asc>; :}
- | DESCENDING {: RESULT = #<desc>; :}
- | {: RESULT = #<asc>; :}
- ;
-opt_limit ::= LIMIT expr:n {: RESULT = n; :}
- | {: RESULT = #<none>; :}
- ;
-opt_groupby ::= GROUP BY groupby_binds:gl
- opt_having:h {: RESULT = #<groupby(`h,...gl)>; :}
- | {: RESULT = #<groupby()>; :}
- ;
-groupby_binds ::= pattern:p COLON expr:e {: RESULT = #[bind(`p,`e)]; :}
- | pattern:p {: RESULT = #[bind(`p,`p)]; :}
- ;
-opt_having ::= HAVING expr:e {: RESULT = e; :}
- | {: RESULT = #<true>; :}
- ;
-type ::= var:v {: RESULT = v; :}
- | LP RP {: RESULT = #<tuple()>; :}
- | LP typel:el RP {: RESULT = #<tuple(...el)>; :}
- | LT type_binds:nl GT {: RESULT = #<record(...nl)>; :}
- | LB type:tp RB {: RESULT = #<list(`tp)>; :}
- | LSB type:tp RSB {: RESULT = #<bag(`tp)>; :}
- | Variable:v LP typel:tl RP {: RESULT = #<`v(...tl)>; :}
- | EXCLAMATION type:tp {: RESULT = #<persistent(`tp)>; :}
- ;
-typel ::= type:e {: RESULT = #[`e]; :}
- | typel:el COMMA type:e {: RESULT = el.append(e); :}
- ;
-type_binds ::= type_binds:nl
- COMMA var:v COLON type:e {: RESULT = nl.append(#<bind(`v,`e)>); :}
- | var:v COLON type:e {: RESULT = #[bind(`v,`e)]; :}
- ;
-data_binds ::= data_binds:nl
- SEP Variable:c COLON type:e {: RESULT = nl.append(#<`c(`e)>); :}
- | Variable:c COLON type:e {: RESULT = #[`c(`e)]; :}
- ;
-xpath ::= DOT {: RESULT = #<dot>; :}
- | xpath:x DIV xname:n {: RESULT = #<child(`n,`x)>; :}
- | xpath:x DIV DIV xname:n {: RESULT = #<descendant(`n,`x)>; :}
- | xpath:x LB Integer:n RB {: RESULT = #<index(`x,`n)>; :}
- | xpath:x LB xpred:p RB {: RESULT = #<predicate(`p,`x)>; :}
- | DIV xname:n {: RESULT = #<child(`n,root)>; :}
- | DIV DIV xname:n {: RESULT = #<descendant(`n,root)>; :}
- | xname:n {: RESULT = #<child(`n,dot)>; :}
- ;
-xname ::= var:v {: RESULT = v; :}
- | Variable:n COLON Variable:v {: RESULT = new VariableLeaf(n+":"+v); :}
- | TIMES {: RESULT = new VariableLeaf("*"); :}
- | ATSYM Variable:v {: RESULT = new VariableLeaf("@"+v); :}
- | ATSYM Variable:n COLON Variable:v {: RESULT = new VariableLeaf("@"+n+":"+v); :}
- | ATSYM TIMES {: RESULT = new VariableLeaf("@*"); :}
- | ATSYM Variable:n COLON TIMES {: RESULT = new VariableLeaf("@"+n+":*"); :}
- ;
-xpred ::= xpath:x EQ const:c {: RESULT = #<eq(`x,`c)>; :}
- | xpath:x {: RESULT = x; :}
- ;
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/core/mrql.lex
----------------------------------------------------------------------
diff --git a/src/main/java/core/mrql.lex b/src/main/java/core/mrql.lex
deleted file mode 100644
index 01d6509..0000000
--- a/src/main/java/core/mrql.lex
+++ /dev/null
@@ -1,186 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql;
-
-import java_cup.runtime.Symbol;
-import org.apache.mrql.gen.Tree;
-
-%%
-%class MRQLLex
-%public
-%line
-%char
-%ignorecase
-%cup
-%state COMMENT
-%state TEMPLATE
-%eofval{
- return symbol(sym.EOF);
-%eofval}
-%{
-
- static int prev_char_pos = -1;
-
- public static int[] nest = new int[1000];
- public static int nest_pos = 0;
-
- static String template = null;
-
- public static void reset () {
- nest_pos = 0;
- nest[0] = 0;
- prev_char_pos = -1;
- }
-
- public static void record_begin () {
- nest[++nest_pos] = 0;
- }
-
- public static void record_end () {
- nest_pos--;
- }
-
- public int line_pos () { return yyline+1; }
-
- public int char_pos () { return yychar-prev_char_pos; }
-
- public Symbol symbol ( int s ) {
- Tree.line_number = line_pos();
- Tree.position_number = char_pos();
- return new Symbol(s);
- }
-
- public Symbol symbol ( int s, Object o ) {
- Tree.line_number = line_pos();
- Tree.position_number = char_pos();
- return new Symbol(s,o);
- }
-
- public void error ( String msg ) {
- System.err.println("*** Scanner Error: " + msg + " (line: " + line_pos() + ", position: " + char_pos() + ")");
- if (Config.testing)
- throw new Error("Scanner Error");
- }
-
- public String format ( String s ) {
- return s.replaceAll("\\\\t", "\t").replaceAll("\\\\n", "\n");
- }
-%}
-
-ID=[a-zA-Z][a-zA-Z0-9_]*
-INT = [0-9]+
-DOUBLE = [0-9]+([\.][0-9]+)?([eE][+-]?[0-9]+)?
-
-%%
-
-<YYINITIAL> {INT} { return symbol(sym.Integer,new Long(yytext())); }
-<YYINITIAL> {DOUBLE} { return symbol(sym.Double,new Double(yytext())); }
-<YYINITIAL> "["{ID}"|" { yybegin(TEMPLATE); template = ""; return symbol(sym.START_TEMPLATE,yytext()); }
-<YYINITIAL> "if" { return symbol(sym.IF); }
-<YYINITIAL> "then" { return symbol(sym.THEN); }
-<YYINITIAL> "else" { return symbol(sym.ELSE); }
-<YYINITIAL> "select" { return symbol(sym.SELECT); }
-<YYINITIAL> "from" { return symbol(sym.FROM); }
-<YYINITIAL> "having" { return symbol(sym.HAVING); }
-<YYINITIAL> "[" { nest[nest_pos]++; return symbol(sym.LB); }
-<YYINITIAL> "]" { nest[nest_pos]--; return symbol(sym.RB); }
-<YYINITIAL> "(" { nest[nest_pos]++; return symbol(sym.LP); }
-<YYINITIAL> ")" { nest[nest_pos]--; return symbol(sym.RP); }
-<YYINITIAL> "{" { nest[nest_pos]++; return symbol(sym.LSB); }
-<YYINITIAL> "}}" { nest[nest_pos]--; if (nest_pos > 0 && nest[nest_pos] == 0) { record_end(); yybegin(TEMPLATE); } else return symbol(sym.RSB); }
-<YYINITIAL> "}" { nest[nest_pos]--; return symbol(sym.RSB); }
-<YYINITIAL> "+" { return symbol(sym.PLUS); }
-<YYINITIAL> "-" { return symbol(sym.MINUS); }
-<YYINITIAL> "*" { return symbol(sym.TIMES); }
-<YYINITIAL> "/" { return symbol(sym.DIV); }
-<YYINITIAL> "mod" { return symbol(sym.MOD); }
-<YYINITIAL> "%" { return symbol(sym.MOD); }
-<YYINITIAL> "=" { return symbol(sym.EQ); }
-<YYINITIAL> "<>" { return symbol(sym.NEQ); }
-<YYINITIAL> "<=" { return symbol(sym.LEQ); }
-<YYINITIAL> "<" { return symbol(sym.LT); }
-<YYINITIAL> ">=" { return symbol(sym.GEQ); }
-<YYINITIAL> "!" { return symbol(sym.EXCLAMATION); }
-<YYINITIAL> ">" { return (nest_pos > 0 && nest[nest_pos] == 0) ? symbol(sym.GTR) : symbol(sym.GT); }
-<YYINITIAL> \\ { return symbol(sym.BSLASH); }
-<YYINITIAL> "and" { return symbol(sym.AND); }
-<YYINITIAL> "or" { return symbol(sym.OR); }
-<YYINITIAL> "not" { return symbol(sym.NOT); }
-<YYINITIAL> "union" { return symbol(sym.UNION); }
-<YYINITIAL> "intersect" { return symbol(sym.INTERSECT); }
-<YYINITIAL> "except" { return symbol(sym.EXCEPT); }
-<YYINITIAL> "exists" { return symbol(sym.EXISTS); }
-<YYINITIAL> "in" { return symbol(sym.IN); }
-<YYINITIAL> "let" { return symbol(sym.LET); }
-<YYINITIAL> "," { return symbol(sym.COMMA); }
-<YYINITIAL> "." { return symbol(sym.DOT); }
-<YYINITIAL> ":=" { return symbol(sym.ASSIGN); }
-<YYINITIAL> ":" { return symbol(sym.COLON); }
-<YYINITIAL> ";" { return symbol(sym.SEMI); }
-<YYINITIAL> "#" { return symbol(sym.SHARP); }
-<YYINITIAL> "@" { return symbol(sym.ATSYM); }
-<YYINITIAL> \| { return symbol(sym.SEP); }
-<YYINITIAL> "where" { return symbol(sym.WHERE); }
-<YYINITIAL> "order" { return symbol(sym.ORDER); }
-<YYINITIAL> "group" { return symbol(sym.GROUP); }
-<YYINITIAL> "by" { return symbol(sym.BY); }
-<YYINITIAL> "asc" { return symbol(sym.ASCENDING); }
-<YYINITIAL> "desc" { return symbol(sym.DESCENDING); }
-<YYINITIAL> "function" { return symbol(sym.FUNCTION); }
-<YYINITIAL> "macro" { return symbol(sym.MACRO); }
-<YYINITIAL> "distinct" { return symbol(sym.DISTINCT); }
-<YYINITIAL> "as" { return symbol(sym.AS); }
-<YYINITIAL> "some" { return symbol(sym.SOME); }
-<YYINITIAL> "all" { return symbol(sym.ALL); }
-<YYINITIAL> "store" { return symbol(sym.STORE); }
-<YYINITIAL> "dump" { return symbol(sym.DUMP); }
-<YYINITIAL> "type" { return symbol(sym.TYPE); }
-<YYINITIAL> "data" { return symbol(sym.DATA); }
-<YYINITIAL> "case" { return symbol(sym.CASE); }
-<YYINITIAL> "xpath" { return symbol(sym.XPATH); }
-<YYINITIAL> "repeat" { return symbol(sym.REPEAT); }
-<YYINITIAL> "step" { return symbol(sym.STEP); }
-<YYINITIAL> "limit" { return symbol(sym.LIMIT); }
-<YYINITIAL> "import" { return symbol(sym.IMPORT); }
-<YYINITIAL> "parser" { return symbol(sym.PARSER); }
-<YYINITIAL> "include" { return symbol(sym.INCLUDE); }
-<YYINITIAL> "aggregation" { return symbol(sym.AGGREGATION); }
-
-<YYINITIAL> {ID} { return symbol(sym.Variable,yytext()); }
-
-<YYINITIAL> "/*" { yybegin(COMMENT); }
-<COMMENT> "*/" { yybegin(YYINITIAL); }
-<COMMENT> [ \t\f] { }
-<COMMENT> [\r\n] { prev_char_pos = yychar; }
-<COMMENT> . { }
-
-<TEMPLATE> "|]" { yybegin(YYINITIAL); String s = template; template = ""; return symbol(sym.END_TEMPLATE,s); }
-<TEMPLATE> "{{" { yybegin(YYINITIAL); record_begin(); nest[nest_pos]++; String s = template; template = ""; return symbol(sym.TEXT,s); }
-<TEMPLATE> [ \t\f] { template += yytext(); }
-<TEMPLATE> [\r\n] { template += yytext(); prev_char_pos = yychar; }
-<TEMPLATE> . { template += yytext(); }
-
-<YYINITIAL> "//"[^\n]*\n { prev_char_pos = yychar; }
-
-<YYINITIAL> \"[^\"]*\" { return symbol(sym.String,format(yytext().substring(1,yytext().length()-1))); }
-<YYINITIAL> \'[^\']*\' { return symbol(sym.String,format(yytext().substring(1,yytext().length()-1))); }
-
-<YYINITIAL> [ \t\f] { }
-<YYINITIAL> [\r\n] { prev_char_pos = yychar; }
-
-<YYINITIAL> . { error("Illegal character: "+yytext()); }
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Condition.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Condition.java b/src/main/java/gen/Condition.java
deleted file mode 100644
index 6b28585..0000000
--- a/src/main/java/gen/Condition.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-
-
-final public class Condition {
- public final String stmt;
- public final String pred;
- public final int unmatched_brackets;
- public Condition ( String s, String p, int n ) {
- stmt = s;
- pred = p;
- unmatched_brackets = n;
- }
- public String and () {
- if (pred.equals("true"))
- return "";
- else return " && " + pred;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Crypt.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Crypt.java b/src/main/java/gen/Crypt.java
deleted file mode 100644
index b6f95a7..0000000
--- a/src/main/java/gen/Crypt.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-abstract public class Crypt {
- public static boolean encryptp = false;
- public static String crypt ( String s ) {
- byte[] b = s.getBytes();
- for (int i=0; i<b.length; i++)
- { if (b[i]<45)
- b[i] = (byte) (b[i]+78);
- else b[i] = (byte) (b[i]-13);
- if (b[i]==34)
- b[i] = 123;
- else if (b[i]==92)
- b[i] = 124;
- else if (b[i]=='\n')
- b[i] = 125;
- };
- return new String(b);
- }
- public static String decrypt ( String s ) {
- byte[] b = s.getBytes();
- for (int i=0; i<b.length; i++)
- { if (b[i]==123)
- b[i] = 47;
- else if (b[i]==124)
- b[i] = 105;
- else if (b[i]==125)
- b[i] = '\n';
- else if (b[i]>109)
- b[i] = (byte) (b[i]-78);
- else b[i] = (byte) (b[i]+13);
- };
- return new String(b);
- }
- public static String quotes ( String s ) { return "\"" + s + "\""; }
- public static String encrypt ( String s ) {
- if (encryptp)
- return "Crypt.decrypt(\"" + crypt(s) + "\")";
- else return quotes(s);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/DoubleLeaf.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/DoubleLeaf.java b/src/main/java/gen/DoubleLeaf.java
deleted file mode 100644
index ba9ac90..0000000
--- a/src/main/java/gen/DoubleLeaf.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-final public class DoubleLeaf extends Tree {
- public double value;
- public DoubleLeaf ( float n ) {
- super();
- value = (double)n;
- }
- public DoubleLeaf ( double n ) {
- super();
- value = n;
- }
- public double value () { return value; }
- public boolean equals ( Tree e ) {
- return (e instanceof DoubleLeaf)
- && value==((DoubleLeaf) e).value;
- }
- protected int size () { return toString().length(); }
- public String toString () {
- return Double.toString(value);
- }
- public String pretty ( int position ) {
- return toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/LongLeaf.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/LongLeaf.java b/src/main/java/gen/LongLeaf.java
deleted file mode 100644
index 6611f32..0000000
--- a/src/main/java/gen/LongLeaf.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-
-
-final public class LongLeaf extends Tree {
- public long value;
-
- public LongLeaf ( int n ) {
- super();
- value = (long)n;
- }
-
- public LongLeaf ( long n ) {
- super();
- value = n;
- }
-
- public long value () { return value; }
-
- public boolean equals ( Tree e ) {
- return (e instanceof LongLeaf)
- && value == ((LongLeaf) e).value;
- }
-
- protected int size () { return toString().length(); }
-
- public String toString () {
- return Long.toString(value);
- }
-
- public String pretty ( int position ) {
- return toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Main.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Main.java b/src/main/java/gen/Main.java
deleted file mode 100644
index 1d4f91e..0000000
--- a/src/main/java/gen/Main.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java_cup.runtime.*;
-import java.io.*;
-import java.util.ArrayList;
-
-public class Main extends GenParser {
- public static void main ( String args[] ) throws Exception {
- ArrayList<String> files = new ArrayList<String>();
- boolean is_directory = false;
- String output = null;
- for ( int i = 0; i < args.length; i++ )
- if (args[i].equals("-encrypt"))
- Crypt.encryptp = true;
- else if (args[i].equals("-o"))
- output = args[++i];
- else files.add(args[i]);
- if (output != null && new File(output).isDirectory())
- is_directory = true;
- if (!is_directory && files.size() > 1)
- throw new Error("Expected an output directory: "+output);
- for ( String file: files )
- try {
- Meta.clear();
- scanner = new GenLex(new FileInputStream(file));
- String outfile = file.replace(".gen",".java");
- if (is_directory)
- outfile = new File(output,new File(outfile).getName()).getPath();
- if (outfile.equals(file))
- if (is_directory) {
- System.err.println("Cannot compile: "+file);
- continue;
- } else outfile = output;
- out = new PrintStream(new FileOutputStream(outfile));
- out.print("/* DO NOT EDIT THIS FILE. THIS FILE WAS GENERATED FROM "+file+" BY GEN */");
- new GenParser(scanner).parse();
- Meta.dump_names(out);
- out.close();
- } catch (Error ex) {
- System.err.println(ex.getMessage()+" while parsing the GEN file: "+file);
- System.exit(-1);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Meta.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Meta.java b/src/main/java/gen/Meta.java
deleted file mode 100644
index 80d31bd..0000000
--- a/src/main/java/gen/Meta.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-import java.util.HashMap;
-import java.util.Random;
-
-
-final public class Meta extends Crypt {
-
- final public static Tree mark = new LongLeaf(-1);
-
- static int level = 0;
-
- private static int name_counter = 0;
-
- public static Tree new_name () {
- return new VariableLeaf("N" + (name_counter++) + "_");
- }
-
- public static Tree escape ( Tree e ) {
- if (Tree.parsed) {
- Tree.line_number = e.line;
- Tree.position_number = e.position;
- };
- return e;
- }
-
- public static void clear () {
- name_counter = 0;
- name_index = 0;
- name_vars.clear();
- package_name = "PatternNames_"+Math.abs(random.nextLong());
- }
-
- public static Tree escape ( String s ) { return new VariableLeaf(s); }
-
- public static Tree escape ( long n ) { return new LongLeaf(n); }
-
- public static Tree escape ( double n ) { return new DoubleLeaf(n); }
-
- private static HashMap<String,Integer> name_vars = new HashMap<String,Integer>(1000);
-
- private static int name_index = 0;
-
- private final static Random random = new Random();
-
- private static String package_name = "PatternNames_"+Math.abs(random.nextLong());
-
- private static String name ( String s ) {
- Integer ns = name_vars.get(s);
- if (ns == null) {
- ns = new Integer(name_index++);
- name_vars.put(s,ns);
- };
- return package_name+".P_"+ns.intValue();
- }
-
- static void dump_names ( PrintStream out ) {
- if (name_vars.isEmpty())
- return;
- out.println("abstract class "+package_name+" {");
- for ( String s: name_vars.keySet() )
- out.println(" final static String P_"+name_vars.get(s)+" = Tree.add("+s+");");
- out.println("}");
- }
-
- public static Trees subst_list ( Tree term, Tree value, Trees es ) {
- Trees res = Trees.nil;
- for (Trees r = es; !r.is_empty(); r = r.tail)
- res = res.cons(subst_expr(term,value,r.head));
- return res.reverse();
- }
-
- public static Tree subst_expr ( Tree term, Tree value, Tree e ) {
- if (e.equals(term))
- return value;
- else if (e instanceof Node)
- return new Node(((Node)e).name,
- subst_list(term,value,((Node)e).children));
- else return e;
- }
-
- public static Tree substitute ( Trees path, Tree e, Tree value ) {
- if (path.is_empty())
- return value;
- else {
- Trees res = Trees.nil;
- for (Trees r = ((Node)e).children; !r.is_empty(); r = r.tail)
- if (r.head.equals(path.head))
- res = res.cons(substitute(path.tail,r.head,value));
- else res = res.cons(r.head);
- return new Node(((Node)e).name,res.reverse());
- }
- }
-
- public static String reify ( Tree e ) {
- if (e instanceof LongLeaf)
- return "new LongLeaf(" + e + ")";
- else if (e instanceof DoubleLeaf)
- return "new DoubleLeaf(" + e + ")";
- else if (e instanceof VariableLeaf)
- if (((VariableLeaf)e).value.equals("_any_"))
- throw new Error("Gen: Cannot use \"_\" (any) in Tree Construction: "+e);
- else return "new VariableLeaf(" + encrypt(((VariableLeaf)e).value) + ")";
- else if (e instanceof StringLeaf)
- return "new StringLeaf(" + e + ")";
- else {
- Node n = (Node) e;
- if (n.name.equals("Node")) {
- String s = "new Node(";
- if (n.children.head instanceof VariableLeaf)
- if (((VariableLeaf)n.children.head).value.equals("_any_"))
- throw new Error("Gen: Cannot use \"_\" (any) in AST node name: "+e);
- else s = s + encrypt(((VariableLeaf)n.children.head).value);
- else {
- Node m = (Node) n.children.head;
- if (m.name.equals("Code"))
- s = s + ((StringLeaf)m.children.head).value;
- else if (m.name.equals("Escape"))
- s = s + ((VariableLeaf)m.children.head).value;
- };
- s = s + ",Trees.nil";
- for (Trees r= n.children.tail; !r.is_empty(); r = r.tail)
- if (r.head instanceof Node && ((Node)r.head).name.equals("Dots")) {
- Node m = (Node) r.head;
- if (m.children.is_empty())
- throw new Error("Gen: Cannot use \"...\" in Tree construction: "+e);
- else if (m.children.head instanceof VariableLeaf)
- s = s + ".append(" + ((VariableLeaf)m.children.head).value + ")";
- else s = s + ".append(" + ((StringLeaf)m.children.head).value + ")";
- } else s = s + ".append(" + reify(r.head) + ")";
- return s + ")";
- } else if (n.name.equals("Code"))
- return "Meta.escape(" + ((StringLeaf)n.children.head).value + ")";
- else if (n.name.equals("Escape"))
- return "Meta.escape(" + ((VariableLeaf)n.children.head).value + ")";
- else if (n.name.equals("Higher"))
- return "Meta.substitute(" + ((VariableLeaf)n.children.head).value
- + ".tail," + ((VariableLeaf)n.children.head).value
- + ".head," + reify(n.children.tail.head) + ")";
- else throw new Error("Gen: Wrong Tree construction: "+e);
- }
- }
-
- public static Condition pattern_list ( Trees args, String v ) {
- if (args.is_empty())
- return new Condition("",v+".tail==null",0);
- else if (args.head instanceof Node && ((Node)args.head).name.equals("Dots")) {
- if (args.tail.is_empty()) {
- Node m = (Node) args.head;
- if (m.children.is_empty())
- return new Condition("","true",0);
- else if (m.children.head instanceof VariableLeaf)
- return new Condition("Trees " + ((VariableLeaf)m.children.head).value + " = " + v + "; ",
- "true",0);
- else if (m.children.head instanceof StringLeaf)
- new Condition("",v + ".head.equals(" + ((StringLeaf)m.children.head).value + ")",0);
- return new Condition("","true",0);
- } else {
- Node m = (Node) args.head;
- if (m.children.is_empty()) {
- Condition rest = pattern_list(args.tail,"R_");
- return new Condition("Trees R_ = " + v + "; for(; R_.tail!=null && !(" + rest.pred
- + "); R_=R_.tail) ; if (R_.tail!=null) { " + rest.stmt,
- "true",rest.unmatched_brackets+1);
- } else if (m.children.head instanceof VariableLeaf) {
- String nm = ((VariableLeaf)m.children.head).value;
- Condition rest = pattern_list(args.tail,nm+"_");
- return new Condition("Trees " + nm + " = Trees.nil; Trees " + nm + "_=" + v + "; "
- + "for(; " + nm + "_.tail!=null && !(" + rest.pred + "); "
- + nm + "_=(FOUND_" + Meta.level + ")?" + nm + "_:" + nm + "_.tail) " + nm + " = " + nm + ".append("
- + nm + "_.head); " + "if (" + nm + "_.tail!=null) { " + rest.stmt,
- "true",rest.unmatched_brackets+1);
- } else if (m.children.head instanceof StringLeaf) {
- Condition rest = pattern_list(args.tail,v);
- return new Condition(rest.stmt,
- v + ".equals(" + ((StringLeaf)m.children.head).value + ")" + rest.and(),
- rest.unmatched_brackets);
- };
- return new Condition("","true",0);
- }
- } else {
- Condition c = pattern(args.head,v+".head");
- Condition rest = pattern_list(args.tail,v+".tail");
- return new Condition(c.stmt + rest.stmt,
- v + ".tail!=null" + c.and() + rest.and(),
- c.unmatched_brackets+rest.unmatched_brackets);
- }
- }
-
- public static Condition pattern ( Tree e, String v ) {
- if (e instanceof LongLeaf)
- return new Condition("", "(" + v + " instanceof LongLeaf) && ((LongLeaf)" + v + ").value==" + e,0);
- else if (e instanceof DoubleLeaf)
- return new Condition("","(" + v + " instanceof DoubleLeaf) && ((DoubleLeaf)" + v + ").value==" + e,0);
- else if (e instanceof VariableLeaf)
- if (((VariableLeaf)e).value.equals("_any_"))
- return new Condition("","true",0);
- else return new Condition("","(" + v + " instanceof VariableLeaf) && ((VariableLeaf)" + v
- + ").value==" + name(encrypt(((VariableLeaf)e).value)),0);
- else if (e instanceof StringLeaf)
- return new Condition("","(" + v + " instanceof StringLeaf) && ((StringLeaf)" + v + ").value.equals(" + e + ")",0);
- else {
- Node n = (Node) e;
- if (n.name.equals("Node")) {
- String p = "(" + v + " instanceof Node)";
- String s = "";
- if (n.children.head instanceof VariableLeaf) {
- if (((VariableLeaf)n.children.head).value!="_any_")
- p = p + " && ((Node)" + v + ").name=="
- + name(encrypt(((VariableLeaf)n.children.head).value));
- } else if (n.children.head instanceof Node)
- if (((Node)n.children.head).name.equals("Escape"))
- s = "String " + ((VariableLeaf)((Node)n.children.head).children.head).value
- + " = ((Node)" + v + ").name; ";
- else throw new Error("Gen: Wrong Tree pattern: "+e);
- Condition c = pattern_list(n.children.tail,"((Node)" + v + ").children");
- return new Condition(s+c.stmt,p+c.and(),c.unmatched_brackets);
- } else if (n.name.equals("Escape"))
- return new Condition("Tree " + ((VariableLeaf)n.children.head).value + " = " + v + "; ",
- "true",0);
- else if (n.name.equals("IS")) {
- String nm = ((VariableLeaf)n.children.head).value;
- Condition c = pattern(n.children.tail.head,nm);
- return new Condition(c.stmt + "Tree " + nm + " = " + v + "; ",c.pred,0);
- } else if (n.name.equals("Code"))
- return new Condition("",v + ".equals(" + ((StringLeaf)n.children.head).value + ")",0);
- else if (n.name.equals("Higher")) {
- String nm = ((VariableLeaf)n.children.head).value;
- Condition c = pattern(n.children.tail.head,nm+"__");
- String s = reify(n.children.tail.head);
- return new Condition("Tree " + nm + "_ = " + v + "; Trees " + nm + " = Trees.nil; Trees STACK_" + nm
- + "_ = new Trees(" + nm + "_); while (!FOUND_" + Meta.level + " && STACK_" + nm + "_.tail!=null) { Tree "
- + nm + "__ = STACK_" + nm + "_.head; STACK_" + nm + "_ = STACK_" + nm + "_.tail; " + nm
- + " = (" + nm + "__==Meta.mark) ? " + nm + ".tail : ((" + nm + "__ instanceof Node) ? "
- + nm + ".cons(" + nm + "__) : " + nm + "); if (" + nm + "__ instanceof Node) STACK_" + nm
- + "_ = ((Node)" + nm + "__).children.append(STACK_" + nm + "_.cons(Meta.mark)); if ("
- + nm + "__!=Meta.mark" + c.and() + ") { if (!(" + nm + "__ instanceof Node)) " + nm + "="
- + nm + ".cons(" + nm + "__); " + nm + " = " + nm + ".reverse(); " + c.stmt,"true",2);
- } else throw new Error("Gen: Wrong Tree pattern: "+e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/Node.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/Node.java b/src/main/java/gen/Node.java
deleted file mode 100644
index 8bc55d4..0000000
--- a/src/main/java/gen/Node.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-
-
-final public class Node extends Tree {
- public String name;
- public Trees children;
-
- public Node ( String name, Trees children ) {
- super();
- this.name = Tree.add(name);
- this.children = children;
- }
-
- public Node ( String name ) {
- super();
- this.name = name;
- children = Trees.nil;
- }
-
- final public String name () { return name; }
-
- final public Trees children () { return children; }
-
- public boolean equals ( Tree e ) {
- return (e instanceof Node)
- && name == ((Node) e).name
- && children.equals(((Node) e).children);
- }
-
- protected int size () {
- return name().length()+children().size();
- }
-
- public String toString () {
- if (Character.isLetter(name.charAt(0))
- || !(children().length()==2))
- return name + children().toString();
- else return "(" + children().head().toString() + name
- + children().tail().head().toString() + ")";
- }
-
- public String pretty ( int position ) {
- if (Character.isLetter(name.charAt(0))
- || !(children().length()==2))
- return name + children().pretty(position+name.length());
- else return "(" + children().head().toString() + name
- + children().tail().head().toString() + ")";
- }
-
- private void writeObject(ObjectOutputStream out) throws IOException {
- out.defaultWriteObject();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- name = Tree.add(name);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-mrql/blob/1adaa71c/src/main/java/gen/StringLeaf.java
----------------------------------------------------------------------
diff --git a/src/main/java/gen/StringLeaf.java b/src/main/java/gen/StringLeaf.java
deleted file mode 100644
index 5aa41c9..0000000
--- a/src/main/java/gen/StringLeaf.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.mrql.gen;
-
-import java.io.*;
-
-
-final public class StringLeaf extends Tree {
- public String value;
-
- public StringLeaf ( String s ) {
- super();
- value = s;
- }
-
- public String value () { return value; }
-
- public boolean equals ( Tree e ) {
- return (e instanceof StringLeaf)
- && value.equals(((StringLeaf) e).value);
- }
-
- protected int size () { return value.length()+2; }
-
- public String toString () {
- return "\"" + value + "\"";
- }
-
- public String pretty ( int position ) {
- return toString();
- }
-}