You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by im...@apache.org on 2015/08/25 18:44:09 UTC

[21/51] [partial] incubator-asterixdb git commit: Change folder structure for Java repackage

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
new file mode 100644
index 0000000..bfa0331
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -0,0 +1,625 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.translator;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.aql.base.Clause;
+import edu.uci.ics.asterix.aql.base.Expression;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
+import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FieldAccessor;
+import edu.uci.ics.asterix.aql.expression.FieldBinding;
+import edu.uci.ics.asterix.aql.expression.ForClause;
+import edu.uci.ics.asterix.aql.expression.Identifier;
+import edu.uci.ics.asterix.aql.expression.LiteralExpr;
+import edu.uci.ics.asterix.aql.expression.Query;
+import edu.uci.ics.asterix.aql.expression.RecordConstructor;
+import edu.uci.ics.asterix.aql.expression.VariableExpr;
+import edu.uci.ics.asterix.aql.expression.WhereClause;
+import edu.uci.ics.asterix.aql.literal.StringLiteral;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+/**
+ * An AQL statement instance is translated into an instance of type CompileX
+ * that has additional fields for use by the AqlTranslator.
+ */
+public class CompiledStatements {
+
+    public static interface ICompiledStatement {
+
+        public Kind getKind();
+    }
+
+    public static class CompiledDatasetDropStatement implements ICompiledStatement {
+        private final String dataverseName;
+        private final String datasetName;
+
+        public CompiledDatasetDropStatement(String dataverseName, String datasetName) {
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.DATASET_DROP;
+        }
+    }
+
+    // added by yasser
+    public static class CompiledCreateDataverseStatement implements ICompiledStatement {
+        private String dataverseName;
+        private String format;
+
+        public CompiledCreateDataverseStatement(String dataverseName, String format) {
+            this.dataverseName = dataverseName;
+            this.format = format;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getFormat() {
+            return format;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.CREATE_DATAVERSE;
+        }
+    }
+
+    public static class CompiledNodeGroupDropStatement implements ICompiledStatement {
+        private String nodeGroupName;
+
+        public CompiledNodeGroupDropStatement(String nodeGroupName) {
+            this.nodeGroupName = nodeGroupName;
+        }
+
+        public String getNodeGroupName() {
+            return nodeGroupName;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.NODEGROUP_DROP;
+        }
+    }
+
+    public static class CompiledIndexDropStatement implements ICompiledStatement {
+        private String dataverseName;
+        private String datasetName;
+        private String indexName;
+
+        public CompiledIndexDropStatement(String dataverseName, String datasetName, String indexName) {
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+            this.indexName = indexName;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        public String getIndexName() {
+            return indexName;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.INDEX_DROP;
+        }
+    }
+
+    public static class CompiledDataverseDropStatement implements ICompiledStatement {
+        private String dataverseName;
+        private boolean ifExists;
+
+        public CompiledDataverseDropStatement(String dataverseName, boolean ifExists) {
+            this.dataverseName = dataverseName;
+            this.ifExists = ifExists;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public boolean getIfExists() {
+            return ifExists;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.DATAVERSE_DROP;
+        }
+    }
+
+    public static class CompiledTypeDropStatement implements ICompiledStatement {
+        private String typeName;
+
+        public CompiledTypeDropStatement(String nodeGroupName) {
+            this.typeName = nodeGroupName;
+        }
+
+        public String getTypeName() {
+            return typeName;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.TYPE_DROP;
+        }
+    }
+
+    public static interface ICompiledDmlStatement extends ICompiledStatement {
+
+        public String getDataverseName();
+
+        public String getDatasetName();
+    }
+
+    public static class CompiledCreateIndexStatement implements ICompiledDmlStatement {
+        private final String indexName;
+        private final String dataverseName;
+        private final String datasetName;
+        private final List<List<String>> keyFields;
+        private final List<IAType> keyTypes;
+        private final boolean isEnforced;
+        private final IndexType indexType;
+
+        // Specific to NGram index.
+        private final int gramLength;
+
+        public CompiledCreateIndexStatement(String indexName, String dataverseName, String datasetName,
+                List<List<String>> keyFields, List<IAType> keyTypes, boolean isEnforced, int gramLength, IndexType indexType) {
+            this.indexName = indexName;
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+            this.keyFields = keyFields;
+            this.keyTypes = keyTypes;
+            this.gramLength = gramLength;
+            this.isEnforced = isEnforced;
+            this.indexType = indexType;
+        }
+
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getIndexName() {
+            return indexName;
+        }
+
+        public List<List<String>> getKeyFields() {
+            return keyFields;
+        }
+
+        public List<IAType> getKeyFieldTypes() {
+            return keyTypes;
+        }
+
+        public IndexType getIndexType() {
+            return indexType;
+        }
+
+        public int getGramLength() {
+            return gramLength;
+        }
+
+        public boolean isEnforced() {
+            return isEnforced;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.CREATE_INDEX;
+        }
+    }
+
+    public static class CompiledLoadFromFileStatement implements ICompiledDmlStatement {
+        private String dataverseName;
+        private String datasetName;
+        private boolean alreadySorted;
+        private String adapter;
+        private Map<String, String> properties;
+
+        public CompiledLoadFromFileStatement(String dataverseName, String datasetName, String adapter,
+                Map<String, String> properties, boolean alreadySorted) {
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+            this.alreadySorted = alreadySorted;
+            this.adapter = adapter;
+            this.properties = properties;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        public boolean alreadySorted() {
+            return alreadySorted;
+        }
+
+        public String getAdapter() {
+            return adapter;
+        }
+
+        public Map<String, String> getProperties() {
+            return properties;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.LOAD;
+        }
+    }
+
+    public static class CompiledInsertStatement implements ICompiledDmlStatement {
+        private final String dataverseName;
+        private final String datasetName;
+        private final Query query;
+        private final int varCounter;
+
+        public CompiledInsertStatement(String dataverseName, String datasetName, Query query, int varCounter) {
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+            this.query = query;
+            this.varCounter = varCounter;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        public int getVarCounter() {
+            return varCounter;
+        }
+
+        public Query getQuery() {
+            return query;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.INSERT;
+        }
+    }
+
+    public static class CompiledConnectFeedStatement implements ICompiledDmlStatement {
+        private String dataverseName;
+        private String feedName;
+        private String datasetName;
+        private String policyName;
+        private Query query;
+        private int varCounter;
+
+        public CompiledConnectFeedStatement(String dataverseName, String feedName, String datasetName,
+                String policyName, Query query, int varCounter) {
+            this.dataverseName = dataverseName;
+            this.feedName = feedName;
+            this.datasetName = datasetName;
+            this.policyName = policyName;
+            this.query = query;
+            this.varCounter = varCounter;
+        }
+
+        @Override
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getFeedName() {
+            return feedName;
+        }
+
+        @Override
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        public int getVarCounter() {
+            return varCounter;
+        }
+
+        public Query getQuery() {
+            return query;
+        }
+
+        public void setQuery(Query query) {
+            this.query = query;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.CONNECT_FEED;
+        }
+
+        public String getPolicyName() {
+            return policyName;
+        }
+    }
+    
+    public static class CompiledSubscribeFeedStatement implements ICompiledDmlStatement {
+
+        private final FeedConnectionRequest request;
+        private Query query;
+        private final int varCounter;
+
+        public CompiledSubscribeFeedStatement(FeedConnectionRequest request, Query query, int varCounter) {
+            this.request = request;
+            this.query = query;
+            this.varCounter = varCounter;
+        }
+
+        @Override
+        public String getDataverseName() {
+            return request.getReceivingFeedId().getDataverse();
+        }
+
+        @Override
+        public String getDatasetName() {
+            return request.getTargetDataset();
+        }
+
+        public int getVarCounter() {
+            return varCounter;
+        }
+
+        public Query getQuery() {
+            return query;
+        }
+
+        public void setQuery(Query query) {
+            this.query = query;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.SUBSCRIBE_FEED;
+        }
+
+    }
+
+
+    public static class CompiledDisconnectFeedStatement implements ICompiledDmlStatement {
+        private String dataverseName;
+        private String datasetName;
+        private String feedName;
+        private Query query;
+        private int varCounter;
+
+        public CompiledDisconnectFeedStatement(String dataverseName, String feedName, String datasetName) {
+            this.dataverseName = dataverseName;
+            this.feedName = feedName;
+            this.datasetName = datasetName;
+        }
+
+        @Override
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        @Override
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        public String getFeedName() {
+            return feedName;
+        }
+
+        public int getVarCounter() {
+            return varCounter;
+        }
+
+        public Query getQuery() {
+            return query;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.DISCONNECT_FEED;
+        }
+
+    }
+
+    public static class CompiledDeleteStatement implements ICompiledDmlStatement {
+        private VariableExpr var;
+        private String dataverseName;
+        private String datasetName;
+        private Expression condition;
+        private int varCounter;
+        private AqlMetadataProvider metadataProvider;
+
+        public CompiledDeleteStatement(VariableExpr var, String dataverseName, String datasetName,
+                Expression condition, int varCounter, AqlMetadataProvider metadataProvider) {
+            this.var = var;
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+            this.condition = condition;
+            this.varCounter = varCounter;
+            this.metadataProvider = metadataProvider;
+        }
+
+        @Override
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        @Override
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public int getVarCounter() {
+            return varCounter;
+        }
+
+        public Expression getCondition() {
+            return condition;
+        }
+
+        public Query getQuery() throws AlgebricksException {
+
+            List<Expression> arguments = new ArrayList<Expression>();
+            String arg = dataverseName == null ? datasetName : dataverseName + "." + datasetName;
+            LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
+            arguments.add(argumentLiteral);
+
+            CallExpr callExpression = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, "dataset", 1),
+                    arguments);
+            List<Clause> clauseList = new ArrayList<Clause>();
+            Clause forClause = new ForClause(var, callExpression);
+            clauseList.add(forClause);
+            Clause whereClause = null;
+            if (condition != null) {
+                whereClause = new WhereClause(condition);
+                clauseList.add(whereClause);
+            }
+
+            Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+            if (dataset == null) {
+                throw new AlgebricksException("Unknown dataset " + datasetName);
+            }
+            String itemTypeName = dataset.getItemTypeName();
+            IAType itemType = metadataProvider.findType(dataset.getDataverseName(), itemTypeName);
+            ARecordType recType = (ARecordType) itemType;
+            String[] fieldNames = recType.getFieldNames();
+            List<FieldBinding> fieldBindings = new ArrayList<FieldBinding>();
+            for (int i = 0; i < fieldNames.length; i++) {
+                FieldAccessor fa = new FieldAccessor(var, new Identifier(fieldNames[i]));
+                FieldBinding fb = new FieldBinding(new LiteralExpr(new StringLiteral(fieldNames[i])), fa);
+                fieldBindings.add(fb);
+            }
+            RecordConstructor rc = new RecordConstructor(fieldBindings);
+
+            FLWOGRExpression flowgr = new FLWOGRExpression(clauseList, rc);
+            Query query = new Query();
+            query.setBody(flowgr);
+            return query;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.DELETE;
+        }
+
+    }
+
+    public static class CompiledCompactStatement implements ICompiledStatement {
+        private final String dataverseName;
+        private final String datasetName;
+
+        public CompiledCompactStatement(String dataverseName, String datasetName) {
+            this.dataverseName = dataverseName;
+            this.datasetName = datasetName;
+        }
+
+        public String getDataverseName() {
+            return dataverseName;
+        }
+
+        public String getDatasetName() {
+            return datasetName;
+        }
+
+        @Override
+        public Kind getKind() {
+            return Kind.COMPACT;
+        }
+    }
+
+    public static class CompiledIndexCompactStatement extends CompiledCompactStatement {
+        private final String indexName;
+        private final List<List<String>> keyFields;
+        private final List<IAType> keyTypes;
+        private final IndexType indexType;
+        private final boolean isEnforced;
+
+        // Specific to NGram index.
+        private final int gramLength;
+
+        public CompiledIndexCompactStatement(String dataverseName, String datasetName, String indexName,
+                List<List<String>> keyFields, List<IAType> keyTypes, boolean isEnforced, int gramLength, IndexType indexType) {
+            super(dataverseName, datasetName);
+            this.indexName = indexName;
+            this.keyFields = keyFields;
+            this.keyTypes = keyTypes;
+            this.gramLength = gramLength;
+            this.indexType = indexType;
+            this.isEnforced = isEnforced;
+        }
+
+        public String getIndexName() {
+            return indexName;
+        }
+
+        public List<List<String>> getKeyFields() {
+            return keyFields;
+        }
+
+        public List<IAType> getKeyTypes() {
+            return keyTypes;
+        }
+
+        public IndexType getIndexType() {
+            return indexType;
+        }
+
+        public int getGramLength() {
+            return gramLength;
+        }
+
+        public boolean isEnforced() {
+            return isEnforced;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-algebra/src/main/java/org/apache/asterix/translator/ConstantHelper.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/ConstantHelper.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/ConstantHelper.java
new file mode 100644
index 0000000..dc75889
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/ConstantHelper.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.translator;
+
+import edu.uci.ics.asterix.aql.base.Literal;
+import edu.uci.ics.asterix.aql.literal.DoubleLiteral;
+import edu.uci.ics.asterix.aql.literal.FloatLiteral;
+import edu.uci.ics.asterix.aql.literal.IntegerLiteral;
+import edu.uci.ics.asterix.aql.literal.LongIntegerLiteral;
+import edu.uci.ics.asterix.aql.literal.StringLiteral;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.ADouble;
+import edu.uci.ics.asterix.om.base.AFloat;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IAObject;
+
+public final class ConstantHelper {
+
+    public static IAObject objectFromLiteral(Literal valLiteral) {
+        switch (valLiteral.getLiteralType()) {
+            case DOUBLE: {
+                DoubleLiteral d = (DoubleLiteral) valLiteral;
+                return new ADouble(d.getValue());
+            }
+            case FALSE: {
+                return ABoolean.FALSE;
+            }
+            case FLOAT: {
+                FloatLiteral fl = (FloatLiteral) valLiteral;
+                return new AFloat(fl.getValue());
+            }
+            case INTEGER: {
+                IntegerLiteral il = (IntegerLiteral) valLiteral;
+                return new AInt32(il.getValue());
+            }
+            case LONG: {
+                LongIntegerLiteral il = (LongIntegerLiteral) valLiteral;
+                return new AInt64(il.getValue());
+            }
+            case NULL: {
+                return ANull.NULL;
+            }
+            case STRING: {
+                StringLiteral sl = (StringLiteral) valLiteral;
+                return new AString(sl.getValue());
+            }
+            case TRUE: {
+                return ABoolean.TRUE;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java
new file mode 100644
index 0000000..986b5ca
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.translator;
+
+import java.util.HashMap;
+
+import edu.uci.ics.asterix.aql.expression.VariableExpr;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.Counter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public final class TranslationContext {
+
+    private Counter varCounter;
+    private HashMap<Integer, LogicalVariable> varEnv = new HashMap<Integer, LogicalVariable>();
+    private boolean topFlwor = true;
+
+    public TranslationContext(Counter varCounter) {
+        this.varCounter = varCounter;
+    }
+
+    public int getVarCounter() {
+        return varCounter.get();
+    }
+
+    public boolean isTopFlwor() {
+        return topFlwor;
+    }
+
+    public void setTopFlwor(boolean b) {
+        topFlwor = b;
+    }
+
+    public LogicalVariable getVar(Integer varId) {
+        return varEnv.get(varId);
+    }
+
+    public LogicalVariable getVar(VariableExpr v) {
+        return varEnv.get(v.getVar().getId());
+    }
+
+    public LogicalVariable newVar(VariableExpr v) {
+        Integer i = v.getVar().getId();
+        if (i > varCounter.get()) {
+            varCounter.set(i);
+        }
+        LogicalVariable var = new LogicalVariable(i);
+        varEnv.put(i, var);
+        return var;
+    }
+
+    public void setVar(VariableExpr v, LogicalVariable var) {
+        varEnv.put(v.getVar().getId(), var);
+    }
+
+    public LogicalVariable newVar() {
+        varCounter.inc();
+        LogicalVariable var = new LogicalVariable(varCounter.get());
+        varEnv.put(varCounter.get(), var);
+        return var;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
new file mode 100644
index 0000000..2819961
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.translator;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public class TranslationException extends AsterixException {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 685960054131778068L;
+
+    public TranslationException() {
+        super();
+    }
+
+    public TranslationException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
new file mode 100644
index 0000000..e6f3d17
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
@@ -0,0 +1,392 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.translator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.aql.expression.OrderedListTypeDefinition;
+import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition;
+import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition.RecordKind;
+import edu.uci.ics.asterix.aql.expression.TypeExpression;
+import edu.uci.ics.asterix.aql.expression.TypeReferenceExpression;
+import edu.uci.ics.asterix.aql.expression.UnorderedListTypeDefinition;
+import edu.uci.ics.asterix.common.annotations.IRecordFieldDataGen;
+import edu.uci.ics.asterix.common.annotations.RecordDataGenAnnotation;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
+import edu.uci.ics.asterix.metadata.entities.Datatype;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.AbstractComplexType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeSignature;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class TypeTranslator {
+
+    public static Map<TypeSignature, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeExpression typeExpr,
+            String typeName, String typeDataverse) throws AlgebricksException, MetadataException {
+        Map<TypeSignature, IAType> typeMap = new HashMap<TypeSignature, IAType>();
+        return computeTypes(mdTxnCtx, typeExpr, typeName, typeDataverse, typeMap);
+    }
+
+    public static Map<TypeSignature, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeExpression typeExpr,
+            String typeName, String typeDataverse, Map<TypeSignature, IAType> typeMap) throws AlgebricksException,
+            MetadataException {
+        Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
+        Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<TypeSignature, List<AbstractCollectionType>>();
+        Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences = new HashMap<TypeSignature, List<TypeSignature>>();
+        firstPass(typeExpr, typeName, typeMap, incompleteFieldTypes, incompleteItemTypes,
+                incompleteTopLevelTypeReferences, typeDataverse);
+        secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences,
+                typeDataverse);
+
+        for (IAType type : typeMap.values())
+            if (type.getTypeTag().isDerivedType())
+                ((AbstractComplexType) type).generateNestedDerivedTypeNames();
+        return typeMap;
+    }
+
+    private static Map<String, BuiltinType> builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
+
+    private static void firstPass(TypeExpression typeExpr, String typeName, Map<TypeSignature, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
+            throws AlgebricksException {
+
+        if (builtinTypeMap.get(typeName) != null) {
+            throw new AlgebricksException("Cannot redefine builtin type " + typeName + " .");
+        }
+        TypeSignature typeSignature = new TypeSignature(typeDataverse, typeName);
+        try {
+            switch (typeExpr.getTypeKind()) {
+                case TYPEREFERENCE: {
+                    TypeReferenceExpression tre = (TypeReferenceExpression) typeExpr;
+                    IAType t = solveTypeReference(new TypeSignature(typeDataverse, tre.getIdent().getValue()), typeMap);
+                    if (t != null) {
+                        typeMap.put(typeSignature, t);
+                    } else {
+                        addIncompleteTopLevelTypeReference(typeName, tre, incompleteTopLevelTypeReferences,
+                                typeDataverse);
+                    }
+                    break;
+                }
+                case RECORD: {
+                    RecordTypeDefinition rtd = (RecordTypeDefinition) typeExpr;
+                    ARecordType recType = computeRecordType(typeSignature, rtd, typeMap, incompleteFieldTypes,
+                            incompleteItemTypes, typeDataverse);
+                    typeMap.put(typeSignature, recType);
+                    break;
+                }
+                case ORDEREDLIST: {
+                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) typeExpr;
+                    AOrderedListType olType = computeOrderedListType(typeSignature, oltd, typeMap, incompleteItemTypes,
+                            incompleteFieldTypes, typeDataverse);
+                    typeMap.put(typeSignature, olType);
+                    break;
+                }
+                case UNORDEREDLIST: {
+                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) typeExpr;
+                    AUnorderedListType ulType = computeUnorderedListType(typeSignature, ultd, typeMap,
+                            incompleteItemTypes, incompleteFieldTypes, typeDataverse);
+                    typeMap.put(typeSignature, ulType);
+                    break;
+                }
+                default: {
+                    throw new IllegalStateException();
+                }
+            }
+        } catch (AsterixException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    private static void secondPass(MetadataTransactionContext mdTxnCtx, Map<TypeSignature, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
+            throws AlgebricksException, MetadataException {
+        // solve remaining top level references
+
+        for (TypeSignature typeSignature : incompleteTopLevelTypeReferences.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
+                    typeSignature.getName());
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + typeSignature);
+            } else
+                t = dt.getDatatype();
+            for (TypeSignature sign : incompleteTopLevelTypeReferences.get(typeSignature)) {
+                typeMap.put(sign, t);
+            }
+        }
+        // solve remaining field type references
+        for (String trefName : incompleteFieldTypes.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeDataverse, trefName);
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + trefName);
+            } else
+                t = dt.getDatatype();
+            Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes.get(trefName);
+            for (ARecordType recType : fieldsToFix.keySet()) {
+                List<Integer> positions = fieldsToFix.get(recType);
+                IAType[] fldTypes = recType.getFieldTypes();
+                for (Integer pos : positions) {
+                    if (fldTypes[pos] == null) {
+                        fldTypes[pos] = t;
+                    } else { // nullable
+                        AUnionType nullableUnion = (AUnionType) fldTypes[pos];
+                        nullableUnion.setTypeAtIndex(t, 1);
+                    }
+                }
+            }
+        }
+
+        // solve remaining item type references
+        for (TypeSignature typeSignature : incompleteItemTypes.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = null;
+            if (MetadataManager.INSTANCE != null) {
+                dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
+                        typeSignature.getName());
+                if (dt == null) {
+                    throw new AlgebricksException("Could not resolve type " + typeSignature);
+                }
+                t = dt.getDatatype();
+            } else {
+                t = typeMap.get(typeSignature);
+            }
+            for (AbstractCollectionType act : incompleteItemTypes.get(typeSignature)) {
+                act.setItemType(t);
+            }
+        }
+    }
+
+    private static AOrderedListType computeOrderedListType(TypeSignature typeSignature, OrderedListTypeDefinition oltd,
+            Map<TypeSignature, IAType> typeMap, Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, String defaultDataverse)
+            throws AsterixException {
+        TypeExpression tExpr = oltd.getItemTypeExpression();
+        String typeName = typeSignature != null ? typeSignature.getName() : null;
+        AOrderedListType aolt = new AOrderedListType(null, typeName);
+        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, aolt, defaultDataverse);
+        return aolt;
+    }
+
+    private static AUnorderedListType computeUnorderedListType(TypeSignature typeSignature,
+            UnorderedListTypeDefinition ultd, Map<TypeSignature, IAType> typeMap,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, String defaulDataverse)
+            throws AsterixException {
+        TypeExpression tExpr = ultd.getItemTypeExpression();
+        String typeName = typeSignature != null ? typeSignature.getName() : null;
+        AUnorderedListType ault = new AUnorderedListType(null, typeName);
+        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, ault, defaulDataverse);
+        return ault;
+    }
+
+    private static void setCollectionItemType(TypeExpression tExpr, Map<TypeSignature, IAType> typeMap,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act,
+            String defaultDataverse) throws AsterixException {
+        switch (tExpr.getTypeKind()) {
+            case ORDEREDLIST: {
+                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
+                IAType t = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+                        defaultDataverse);
+                act.setItemType(t);
+                break;
+            }
+            case UNORDEREDLIST: {
+                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
+                IAType t = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+                        defaultDataverse);
+                act.setItemType(t);
+                break;
+            }
+            case RECORD: {
+                RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
+                IAType t = computeRecordType(null, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes,
+                        defaultDataverse);
+                act.setItemType(t);
+                break;
+            }
+            case TYPEREFERENCE: {
+                TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
+                TypeSignature signature = new TypeSignature(defaultDataverse, tre.getIdent().getValue());
+                IAType tref = solveTypeReference(signature, typeMap);
+                if (tref != null) {
+                    act.setItemType(tref);
+                } else {
+                    addIncompleteCollectionTypeReference(act, tre, incompleteItemTypes, defaultDataverse);
+                }
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+    private static void addIncompleteCollectionTypeReference(AbstractCollectionType collType,
+            TypeReferenceExpression tre, Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            String defaultDataverse) {
+        String typeName = tre.getIdent().getValue();
+        TypeSignature typeSignature = new TypeSignature(defaultDataverse, typeName);
+        List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeSignature);
+        if (typeList == null) {
+            typeList = new LinkedList<AbstractCollectionType>();
+            incompleteItemTypes.put(typeSignature, typeList);
+        }
+        typeList.add(collType);
+    }
+
+    private static void addIncompleteFieldTypeReference(ARecordType recType, int fldPosition,
+            TypeReferenceExpression tre, Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+        String typeName = tre.getIdent().getValue();
+        Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes.get(typeName);
+        if (refMap == null) {
+            refMap = new HashMap<ARecordType, List<Integer>>();
+            incompleteFieldTypes.put(typeName, refMap);
+        }
+        List<Integer> typeList = refMap.get(recType);
+        if (typeList == null) {
+            typeList = new ArrayList<Integer>();
+            refMap.put(recType, typeList);
+        }
+        typeList.add(fldPosition);
+    }
+
+    private static void addIncompleteTopLevelTypeReference(String tdeclName, TypeReferenceExpression tre,
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String defaultDataverse) {
+        String name = tre.getIdent().getValue();
+        TypeSignature typeSignature = new TypeSignature(defaultDataverse, name);
+        List<TypeSignature> refList = incompleteTopLevelTypeReferences.get(name);
+        if (refList == null) {
+            refList = new LinkedList<TypeSignature>();
+            incompleteTopLevelTypeReferences.put(new TypeSignature(defaultDataverse, tre.getIdent().getValue()),
+                    refList);
+        }
+        refList.add(typeSignature);
+    }
+
+    private static IAType solveTypeReference(TypeSignature typeSignature, Map<TypeSignature, IAType> typeMap) {
+        IAType builtin = builtinTypeMap.get(typeSignature.getName());
+        if (builtin != null) {
+            return builtin;
+        } else {
+            return typeMap.get(typeSignature);
+        }
+    }
+
+    private static ARecordType computeRecordType(TypeSignature typeSignature, RecordTypeDefinition rtd,
+            Map<TypeSignature, IAType> typeMap, Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes, String defaultDataverse)
+            throws AsterixException {
+        List<String> names = rtd.getFieldNames();
+        int n = names.size();
+        String[] fldNames = new String[n];
+        IAType[] fldTypes = new IAType[n];
+        int i = 0;
+        for (String s : names) {
+            fldNames[i++] = s;
+        }
+        boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
+        ARecordType recType;
+        try {
+            recType = new ARecordType(typeSignature == null ? null : typeSignature.getName(), fldNames, fldTypes,
+                    isOpen);
+        } catch (HyracksDataException e) {
+            throw new AsterixException(e);
+        }
+
+        List<IRecordFieldDataGen> fieldDataGen = rtd.getFieldDataGen();
+        if (fieldDataGen.size() == n) {
+            IRecordFieldDataGen[] rfdg = new IRecordFieldDataGen[n];
+            rfdg = fieldDataGen.toArray(rfdg);
+            recType.getAnnotations().add(new RecordDataGenAnnotation(rfdg, rtd.getUndeclaredFieldsDataGen()));
+        }
+
+        for (int j = 0; j < n; j++) {
+            TypeExpression texpr = rtd.getFieldTypes().get(j);
+            switch (texpr.getTypeKind()) {
+                case TYPEREFERENCE: {
+                    TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+                    TypeSignature signature = new TypeSignature(defaultDataverse, tre.getIdent().getValue());
+                    IAType tref = solveTypeReference(signature, typeMap);
+                    if (tref != null) {
+                        if (!rtd.getNullableFields().get(j)) { // not nullable
+                            fldTypes[j] = tref;
+                        } else { // nullable
+                            fldTypes[j] = AUnionType.createNullableType(tref);
+                        }
+                    } else {
+                        addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
+                        if (rtd.getNullableFields().get(j)) {
+                            fldTypes[j] = AUnionType.createNullableType(null);
+                        }
+                    }
+                    break;
+                }
+                case RECORD: {
+                    RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
+                    IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes,
+                            incompleteItemTypes, defaultDataverse);
+                    if (!rtd.getNullableFields().get(j)) { // not nullable
+                        fldTypes[j] = t2;
+                    } else { // nullable
+                        fldTypes[j] = AUnionType.createNullableType(t2);
+                    }
+                    break;
+                }
+                case ORDEREDLIST: {
+                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+                    IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+                            defaultDataverse);
+                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? AUnionType.createNullableType(t2) : t2;
+                    break;
+                }
+                case UNORDEREDLIST: {
+                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+                    IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes,
+                            incompleteFieldTypes, defaultDataverse);
+                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? AUnionType.createNullableType(t2) : t2;
+                    break;
+                }
+                default: {
+                    throw new IllegalStateException();
+                }
+            }
+
+        }
+
+        return recType;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
deleted file mode 100644
index 8ccc9d5..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ /dev/null
@@ -1,411 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.api.common;
-
-import java.io.PrintWriter;
-import java.rmi.RemoteException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.json.JSONException;
-
-import edu.uci.ics.asterix.api.common.Job.SubmissionMode;
-import edu.uci.ics.asterix.aql.base.Statement.Kind;
-import edu.uci.ics.asterix.aql.expression.FunctionDecl;
-import edu.uci.ics.asterix.aql.expression.Query;
-import edu.uci.ics.asterix.aql.expression.visitor.AQLPrintVisitor;
-import edu.uci.ics.asterix.aql.rewrites.AqlRewriter;
-import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
-import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
-import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.dataflow.data.common.AqlExpressionTypeComputer;
-import edu.uci.ics.asterix.dataflow.data.common.AqlMergeAggregationExpressionFactory;
-import edu.uci.ics.asterix.dataflow.data.common.AqlNullableTypeComputer;
-import edu.uci.ics.asterix.dataflow.data.common.AqlPartialAggregationTypeComputer;
-import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.jobgen.AqlLogicalExpressionJobGen;
-import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
-import edu.uci.ics.asterix.optimizer.base.RuleCollections;
-import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
-import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
-import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
-import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
-import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
-import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPlotter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-
-/**
- * Provides helper methods for compilation of a query into a JobSpec and submission
- * to Hyracks through the Hyracks client interface.
- */
-public class APIFramework {
-
-    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultLogicalRewrites() {
-        List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultLogicalRewrites = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
-        SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
-        SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
-        SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
-                RuleCollections.buildInitialTranslationRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
-                RuleCollections.buildTypeInferenceRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
-                RuleCollections.buildAutogenerateIDRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
-                RuleCollections.buildNormalizationRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
-                RuleCollections.buildCondPushDownAndJoinInferenceRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
-                RuleCollections.buildLoadFieldsRuleCollection()));
-        // fj
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
-                RuleCollections.buildFuzzyJoinRuleCollection()));
-        //
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
-                RuleCollections.buildNormalizationRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
-                RuleCollections.buildCondPushDownAndJoinInferenceRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
-                RuleCollections.buildLoadFieldsRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
-                RuleCollections.buildDataExchangeRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
-                RuleCollections.buildConsolidationRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
-                RuleCollections.buildAccessMethodRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
-                RuleCollections.buildPlanCleanupRuleCollection()));
-
-        //put TXnRuleCollection!
-        return defaultLogicalRewrites;
-    }
-
-    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultPhysicalRewrites() {
-        List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultPhysicalRewrites = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
-        SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
-        SequentialOnceRuleController seqOnceTopLevel = new SequentialOnceRuleController(false);
-        defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
-                RuleCollections.buildPhysicalRewritesAllLevelsRuleCollection()));
-        defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceTopLevel,
-                RuleCollections.buildPhysicalRewritesTopLevelRuleCollection()));
-        defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
-                RuleCollections.prepareForJobGenRuleCollection()));
-        return defaultPhysicalRewrites;
-    }
-
-    private static class AqlOptimizationContextFactory implements IOptimizationContextFactory {
-
-        public static final AqlOptimizationContextFactory INSTANCE = new AqlOptimizationContextFactory();
-
-        private AqlOptimizationContextFactory() {
-        }
-
-        @Override
-        public IOptimizationContext createOptimizationContext(int varCounter,
-                IExpressionEvalSizeComputer expressionEvalSizeComputer,
-                IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
-                IExpressionTypeComputer expressionTypeComputer, INullableTypeComputer nullableTypeComputer,
-                PhysicalOptimizationConfig physicalOptimizationConfig) {
-            return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
-                    mergeAggregationExpressionFactory, expressionTypeComputer, nullableTypeComputer,
-                    physicalOptimizationConfig);
-        }
-
-    }
-
-    public static Pair<Query, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            AqlMetadataProvider metadataProvider, Query q, SessionConfig conf) throws AsterixException {
-
-        if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_EXPR_TREE)) {
-            conf.out().println();
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("<h4>Expression tree:</h4>");
-                conf.out().println("<pre>");
-            } else {
-                conf.out().println("----------Expression tree:");
-            }
-
-            if (q != null) {
-                q.accept(new AQLPrintVisitor(conf.out()), 0);
-            }
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("</pre>");
-            }
-        }
-        AqlRewriter rw = new AqlRewriter(declaredFunctions, q, metadataProvider);
-        rw.rewrite();
-        Query rwQ = rw.getExpr();
-        return new Pair(rwQ, rw.getVarCounter());
-    }
-
-    public static JobSpecification compileQuery(List<FunctionDecl> declaredFunctions,
-            AqlMetadataProvider queryMetadataProvider, Query rwQ, int varCounter, String outputDatasetName,
-            SessionConfig conf, ICompiledDmlStatement statement) throws AsterixException, AlgebricksException,
-            JSONException, RemoteException, ACIDException {
-
-        if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
-            conf.out().println();
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("<h4>Rewritten expression tree:</h4>");
-                conf.out().println("<pre>");
-            } else {
-                conf.out().println("----------Rewritten expression:");
-            }
-
-            if (rwQ != null) {
-                rwQ.accept(new AQLPrintVisitor(conf.out()), 0);
-            }
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("</pre>");
-            }
-        }
-
-        edu.uci.ics.asterix.common.transactions.JobId asterixJobId = JobIdFactory.generateJobId();
-        queryMetadataProvider.setJobId(asterixJobId);
-        AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(queryMetadataProvider, varCounter,
-                outputDatasetName, statement);
-
-        ILogicalPlan plan;
-        // statement = null when it's a query
-        if (statement == null || statement.getKind() != Kind.LOAD) {
-            plan = t.translate(rwQ);
-        } else {
-            plan = t.translateLoad();
-        }
-
-        LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
-        if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
-            conf.out().println();
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("<h4>Logical plan:</h4>");
-                conf.out().println("<pre>");
-            } else {
-                conf.out().println("----------Logical plan:");
-            }
-
-            if (rwQ != null || statement.getKind() == Kind.LOAD) {
-                StringBuilder buffer = new StringBuilder();
-                PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
-                conf.out().print(buffer);
-            }
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("</pre>");
-            }
-        }
-
-        //print the plot for the logical plan
-        AsterixExternalProperties xProps = AsterixAppContextInfo.getInstance().getExternalProperties();
-        Boolean plot = xProps.getIsPlottingEnabled();
-        if (plot) {
-            PlanPlotter.printLogicalPlan(plan);
-        }
-
-        AsterixCompilerProperties compilerProperties = AsterixAppContextInfo.getInstance().getCompilerProperties();
-        int frameSize = compilerProperties.getFrameSize();
-        int sortFrameLimit = (int) (compilerProperties.getSortMemorySize() / frameSize);
-        int groupFrameLimit = (int) (compilerProperties.getGroupMemorySize() / frameSize);
-        int joinFrameLimit = (int) (compilerProperties.getJoinMemorySize() / frameSize);
-        OptimizationConfUtil.getPhysicalOptimizationConfig().setFrameSize(frameSize);
-        OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalSort(sortFrameLimit);
-        OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalGroupBy(groupFrameLimit);
-        OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesHybridHash(joinFrameLimit);
-
-        HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder(
-                AqlOptimizationContextFactory.INSTANCE);
-        builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
-        builder.setLogicalRewrites(buildDefaultLogicalRewrites());
-        builder.setPhysicalRewrites(buildDefaultPhysicalRewrites());
-        IDataFormat format = queryMetadataProvider.getFormat();
-        ICompilerFactory compilerFactory = builder.create();
-        builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
-        builder.setIMergeAggregationExpressionFactory(new AqlMergeAggregationExpressionFactory());
-        builder.setPartialAggregationTypeComputer(new AqlPartialAggregationTypeComputer());
-        builder.setExpressionTypeComputer(AqlExpressionTypeComputer.INSTANCE);
-        builder.setNullableTypeComputer(AqlNullableTypeComputer.INSTANCE);
-
-        ICompiler compiler = compilerFactory.createCompiler(plan, queryMetadataProvider, t.getVarCounter());
-        if (conf.isOptimize()) {
-            compiler.optimize();
-            //plot optimized logical plan
-            if (plot)
-                PlanPlotter.printOptimizedLogicalPlan(plan);
-            if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
-                if (conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)) {
-                    // For Optimizer tests.
-                    StringBuilder buffer = new StringBuilder();
-                    PlanPrettyPrinter.printPhysicalOps(plan, buffer, 0);
-                    conf.out().print(buffer);
-                } else {
-                    if (conf.is(SessionConfig.FORMAT_HTML)) {
-                        conf.out().println("<h4>Optimized logical plan:</h4>");
-                        conf.out().println("<pre>");
-                    } else {
-                        conf.out().println("----------Optimized logical plan:");
-                    }
-
-                    if (rwQ != null || statement.getKind() == Kind.LOAD) {
-                        StringBuilder buffer = new StringBuilder();
-                        PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
-                        conf.out().print(buffer);
-                    }
-
-                    if (conf.is(SessionConfig.FORMAT_HTML)) {
-                        conf.out().println("</pre>");
-                    }
-                }
-            }
-        }
-
-        if (!conf.isGenerateJobSpec()) {
-            return null;
-        }
-
-        AlgebricksPartitionConstraint clusterLocs = queryMetadataProvider.getClusterLocations();
-        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
-        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
-        builder.setClusterLocations(clusterLocs);
-        builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
-        builder.setExpressionRuntimeProvider(new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(
-                AqlLogicalExpressionJobGen.INSTANCE));
-        builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
-        builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
-        builder.setNullWriterFactory(format.getNullWriterFactory());
-        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
-
-        switch (conf.fmt()) {
-            case JSON:
-                builder.setPrinterProvider(format.getJSONPrinterFactoryProvider());
-                break;
-            case CSV:
-                builder.setPrinterProvider(format.getCSVPrinterFactoryProvider());
-                break;
-            case ADM:
-                builder.setPrinterProvider(format.getPrinterFactoryProvider());
-                break;
-            default:
-                throw new RuntimeException("Unexpected OutputFormat!");
-        }
-
-        builder.setSerializerDeserializerProvider(format.getSerdeProvider());
-        builder.setTypeTraitProvider(format.getTypeTraitProvider());
-        builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
-
-        JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(asterixJobId,
-                queryMetadataProvider.isWriteTransaction());
-        JobSpecification spec = compiler.createJob(AsterixAppContextInfo.getInstance(), jobEventListenerFactory);
-
-        if (conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("<h4>Hyracks job:</h4>");
-                conf.out().println("<pre>");
-            } else {
-                conf.out().println("----------Hyracks job:");
-            }
-
-            if (rwQ != null) {
-                conf.out().println(spec.toJSON().toString(1));
-                conf.out().println(spec.getUserConstraints());
-            }
-
-            if (conf.is(SessionConfig.FORMAT_HTML)) {
-                conf.out().println("</pre>");
-            }
-        }
-        return spec;
-    }
-
-    public static void executeJobArray(IHyracksClientConnection hcc, JobSpecification[] specs, PrintWriter out)
-            throws Exception {
-        for (int i = 0; i < specs.length; i++) {
-            specs[i].setMaxReattempts(0);
-            JobId jobId = hcc.startJob(specs[i]);
-            long startTime = System.currentTimeMillis();
-            hcc.waitForCompletion(jobId);
-            long endTime = System.currentTimeMillis();
-            double duration = (endTime - startTime) / 1000.00;
-            out.println("<pre>Duration: " + duration + " sec</pre>");
-        }
-
-    }
-
-    public static void executeJobArray(IHyracksClientConnection hcc, Job[] jobs, PrintWriter out) throws Exception {
-        for (int i = 0; i < jobs.length; i++) {
-            jobs[i].getJobSpec().setMaxReattempts(0);
-            long startTime = System.currentTimeMillis();
-            try {
-                JobId jobId = hcc.startJob(jobs[i].getJobSpec());
-                if (jobs[i].getSubmissionMode() == SubmissionMode.ASYNCHRONOUS) {
-                    continue;
-                }
-                hcc.waitForCompletion(jobId);
-            } catch (Exception e) {
-                e.printStackTrace();
-                continue;
-            }
-            long endTime = System.currentTimeMillis();
-            double duration = (endTime - startTime) / 1000.00;
-            out.println("<pre>Duration: " + duration + " sec</pre>");
-        }
-
-    }
-
-    private static IDataFormat getDataFormat(MetadataTransactionContext mdTxnCtx, String dataverseName)
-            throws AsterixException {
-        Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
-        IDataFormat format;
-        try {
-            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-        return format;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
deleted file mode 100644
index 6d7f2a4..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ /dev/null
@@ -1,271 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.api.common;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.logging.Logger;
-
-import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
-import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
-import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
-import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
-import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixFileMapManager;
-import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
-import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.feeds.FeedManager;
-import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
-import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
-import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponentManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
-import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
-import edu.uci.ics.hyracks.storage.common.buffercache.DelayPageCleanerPolicy;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IPageCleanerPolicy;
-import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
-import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
-import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
-import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
-
-public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
-
-    private static final AsterixPropertiesAccessor ASTERIX_PROPERTIES_ACCESSOR;
-
-    static {
-        try {
-            ASTERIX_PROPERTIES_ACCESSOR = new AsterixPropertiesAccessor();
-        } catch (AsterixException e) {
-            throw new ExceptionInInitializerError(e);
-        }
-    }
-
-    private static final int METADATA_IO_DEVICE_ID = 0;
-
-    private ILSMMergePolicyFactory metadataMergePolicyFactory;
-    private final INCApplicationContext ncApplicationContext;
-
-    private AsterixCompilerProperties compilerProperties;
-    private AsterixExternalProperties externalProperties;
-    private AsterixMetadataProperties metadataProperties;
-    private AsterixStorageProperties storageProperties;
-    private AsterixTransactionProperties txnProperties;
-    private AsterixFeedProperties feedProperties;
-
-
-    private AsterixThreadExecutor threadExecutor;
-    private DatasetLifecycleManager indexLifecycleManager;
-    private IFileMapManager fileMapManager;
-    private IBufferCache bufferCache;
-    private ITransactionSubsystem txnSubsystem;
-
-    private ILSMIOOperationScheduler lsmIOScheduler;
-    private ILocalResourceRepository localResourceRepository;
-    private ResourceIdFactory resourceIdFactory;
-    private IIOManager ioManager;
-    private boolean isShuttingdown;
-
-    private IFeedManager feedManager;
-
-    public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) throws AsterixException {
-        this.ncApplicationContext = ncApplicationContext;
-        compilerProperties = new AsterixCompilerProperties(ASTERIX_PROPERTIES_ACCESSOR);
-        externalProperties = new AsterixExternalProperties(ASTERIX_PROPERTIES_ACCESSOR);
-        metadataProperties = new AsterixMetadataProperties(ASTERIX_PROPERTIES_ACCESSOR);
-        storageProperties = new AsterixStorageProperties(ASTERIX_PROPERTIES_ACCESSOR);
-        txnProperties = new AsterixTransactionProperties(ASTERIX_PROPERTIES_ACCESSOR);
-        feedProperties = new AsterixFeedProperties(ASTERIX_PROPERTIES_ACCESSOR);
-    }
-
-    public void initialize() throws IOException, ACIDException, AsterixException {
-        Logger.getLogger("edu.uci.ics").setLevel(externalProperties.getLogLevel());
-
-        threadExecutor = new AsterixThreadExecutor(ncApplicationContext.getThreadFactory());
-        fileMapManager = new AsterixFileMapManager();
-        ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-        IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
-        ioManager = ncApplicationContext.getRootContext().getIOManager();
-        IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator,
-                storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages());
-        bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
-                storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
-
-        AsynchronousScheduler.INSTANCE.init(ncApplicationContext.getThreadFactory());
-        lsmIOScheduler = AsynchronousScheduler.INSTANCE;
-
-        metadataMergePolicyFactory = new PrefixMergePolicyFactory();
-
-        ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
-                ioManager);
-        localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
-                .createRepository();
-        resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
-
-        IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProdiverForRecovery(
-                this);
-        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider,
-                txnProperties);
-        
-        indexLifecycleManager = new DatasetLifecycleManager(storageProperties, localResourceRepository,
-                MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID,(LogManager)txnSubsystem.getLogManager());
-        
-        isShuttingdown = false;
-
-        feedManager = new FeedManager(ncApplicationContext.getNodeId(), feedProperties,
-                compilerProperties.getFrameSize());
-
-        // The order of registration is important. The buffer cache must registered before recovery and transaction managers.
-        ILifeCycleComponentManager lccm = ncApplicationContext.getLifeCycleComponentManager();
-        lccm.register((ILifeCycleComponent) bufferCache);
-        lccm.register((ILifeCycleComponent) txnSubsystem.getTransactionManager());
-        lccm.register((ILifeCycleComponent) txnSubsystem.getLogManager());
-        lccm.register((ILifeCycleComponent) indexLifecycleManager);
-        lccm.register((ILifeCycleComponent) txnSubsystem.getLockManager());
-        lccm.register((ILifeCycleComponent) txnSubsystem.getRecoveryManager());
-    }
-
-    public boolean isShuttingdown() {
-        return isShuttingdown;
-    }
-
-    public void setShuttingdown(boolean isShuttingdown) {
-        this.isShuttingdown = isShuttingdown;
-    }
-
-    public void deinitialize() throws HyracksDataException {
-    }
-
-    public IBufferCache getBufferCache() {
-        return bufferCache;
-    }
-
-    public IFileMapProvider getFileMapManager() {
-        return fileMapManager;
-    }
-
-    public ITransactionSubsystem getTransactionSubsystem() {
-        return txnSubsystem;
-    }
-
-    public IIndexLifecycleManager getIndexLifecycleManager() {
-        return indexLifecycleManager;
-    }
-
-    public double getBloomFilterFalsePositiveRate() {
-        return storageProperties.getBloomFilterFalsePositiveRate();
-    }
-
-    public ILSMIOOperationScheduler getLSMIOScheduler() {
-        return lsmIOScheduler;
-    }
-
-    public ILocalResourceRepository getLocalResourceRepository() {
-        return localResourceRepository;
-    }
-
-    public ResourceIdFactory getResourceIdFactory() {
-        return resourceIdFactory;
-    }
-
-    public IIOManager getIOManager() {
-        return ioManager;
-    }
-
-    public int getMetaDataIODeviceId() {
-        return METADATA_IO_DEVICE_ID;
-    }
-
-    @Override
-    public AsterixStorageProperties getStorageProperties() {
-        return storageProperties;
-    }
-
-    @Override
-    public AsterixTransactionProperties getTransactionProperties() {
-        return txnProperties;
-    }
-
-    @Override
-    public AsterixCompilerProperties getCompilerProperties() {
-        return compilerProperties;
-    }
-
-    @Override
-    public AsterixMetadataProperties getMetadataProperties() {
-        return metadataProperties;
-    }
-
-    @Override
-    public AsterixExternalProperties getExternalProperties() {
-        return externalProperties;
-    }
-    
-    @Override
-    public AsterixFeedProperties getFeedProperties() {
-        return feedProperties;
-    }
-
-    @Override
-    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
-        return indexLifecycleManager.getVirtualBufferCaches(datasetID);
-    }
-
-    @Override
-    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
-        return indexLifecycleManager.getOperationTracker(datasetID);
-    }
-
-    @Override
-    public AsterixThreadExecutor getThreadExecutor() {
-        return threadExecutor;
-    }
-
-    public ILSMMergePolicyFactory getMetadataMergePolicyFactory() {
-        return metadataMergePolicyFactory;
-    }
-
-    @Override
-    public IFeedManager getFeedManager() {
-        return feedManager;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/34d81630/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProdiverForRecovery.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProdiverForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProdiverForRecovery.java
deleted file mode 100644
index 938a8b9..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProdiverForRecovery.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.api.common;
-
-import java.util.List;
-
-import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
-import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
-import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
-
-public class AsterixAppRuntimeContextProdiverForRecovery implements IAsterixAppRuntimeContextProvider {
-
-    private final AsterixAppRuntimeContext asterixAppRuntimeContext;
-
-    public AsterixAppRuntimeContextProdiverForRecovery(AsterixAppRuntimeContext asterixAppRuntimeContext) {
-        this.asterixAppRuntimeContext = asterixAppRuntimeContext;
-    }
-
-    @Override
-    public IBufferCache getBufferCache() {
-        return asterixAppRuntimeContext.getBufferCache();
-    }
-
-    @Override
-    public IFileMapProvider getFileMapManager() {
-        return asterixAppRuntimeContext.getFileMapManager();
-    }
-
-    @Override
-    public ITransactionSubsystem getTransactionSubsystem() {
-        return asterixAppRuntimeContext.getTransactionSubsystem();
-    }
-
-    @Override
-    public IIndexLifecycleManager getIndexLifecycleManager() {
-        return asterixAppRuntimeContext.getIndexLifecycleManager();
-    }
-
-    @Override
-    public double getBloomFilterFalsePositiveRate() {
-        return asterixAppRuntimeContext.getBloomFilterFalsePositiveRate();
-    }
-
-    @Override
-    public ILSMIOOperationScheduler getLSMIOScheduler() {
-        return asterixAppRuntimeContext.getLSMIOScheduler();
-    }
-
-    @Override
-    public ILocalResourceRepository getLocalResourceRepository() {
-        return asterixAppRuntimeContext.getLocalResourceRepository();
-    }
-
-    @Override
-    public ResourceIdFactory getResourceIdFactory() {
-        return asterixAppRuntimeContext.getResourceIdFactory();
-    }
-
-    @Override
-    public IIOManager getIOManager() {
-        return asterixAppRuntimeContext.getIOManager();
-    }
-
-    @Override
-    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
-        return asterixAppRuntimeContext.getVirtualBufferCaches(datasetID);
-    }
-
-    @Override
-    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
-        return asterixAppRuntimeContext.getLSMBTreeOperationTracker(datasetID);
-    }
-
-    @Override
-    public IAsterixAppRuntimeContext getAppContext() {
-        return asterixAppRuntimeContext;
-    }
-
-    @Override
-    public AsterixThreadExecutor getThreadExecutor() {
-        return asterixAppRuntimeContext.getThreadExecutor();
-    }
-}