Merge asterix_stabilization into asterix_stabilization_result_distribution.
git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_stabilization_result_distribution@1223 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index fdf9c0b..762fa8f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -83,8 +83,8 @@
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
-import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
-import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.declared.ResultSetDataSink;
+import edu.uci.ics.asterix.metadata.declared.ResultSetSinkId;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
@@ -197,10 +197,12 @@
outputFileSplit = getDefaultOutputFileLocation();
}
metadataProvider.setOutputFile(outputFileSplit);
+ String resultNodeName = outputFileSplit.getNodeName();
+
List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
- FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
- FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+ ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId(), resultNodeName);
+ ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
topOp = new WriteOperator(writeExprList, sink);
topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
} else {
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index 7d6e64a..5af59fd 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -139,6 +139,11 @@
<artifactId>algebricks-compiler</artifactId>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-client</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-aql</artifactId>
<version>0.0.4-SNAPSHOT</version>
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
index 1eb37cd..ae70eb6 100644
--- 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
@@ -146,7 +146,8 @@
public enum DisplayFormat {
TEXT,
- HTML
+ HTML,
+ JSON
}
public static Pair<Query, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
@@ -320,7 +321,7 @@
if (!pc.isGenerateJobSpec()) {
return null;
}
-
+
AlgebricksPartitionConstraint clusterLocs = queryMetadataProvider.getClusterLocations();
builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
index 034e1f4..9915f7c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
@@ -18,7 +18,6 @@
import edu.uci.ics.asterix.aql.parser.AQLParser;
import edu.uci.ics.asterix.aql.parser.ParseException;
import edu.uci.ics.asterix.aql.translator.AqlTranslator;
-import edu.uci.ics.asterix.aql.translator.QueryResult;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
@@ -28,6 +27,283 @@
private static final String HYRACKS_CONNECTION_ATTR = "edu.uci.ics.asterix.HYRACKS_CONNECTION";
+ private static final String HTML_HEADER_TEMPLATE = "<!DOCTYPE html>"
+ + "<html lang=\"en\">"
+ + "<head>"
+ + "<meta name=\"description\" content=\"ASTERIX WEB PAGE\" />"
+ + "<meta name=\"viewport\" content=\"width=device-width, initial-scale=1.0\">"
+ + "<link href='http://fonts.googleapis.com/css?family=Bitter|PT+Sans+Caption|Open+Sans' rel='stylesheet' type='text/css'>"
+ + "<script src=\"http://code.jquery.com/jquery.min.js\"></script>"
+ + "<script src=\"http://www.jacklmoore.com/autosize/jquery.autosize.js\"></script>"
+ + " "
+ + "<link href=\"http://twitter.github.com/bootstrap/assets/css/bootstrap.css\" rel=\"stylesheet\" type=\"text/css\" />"
+ + "<link href=\"http://twitter.github.com/bootstrap/assets/css/bootstrap-responsive.css\" rel=\"stylesheet\">"
+ + ""
+ + " "
+ + "<script src=\"http://twitter.github.com/bootstrap/assets/js/bootstrap.js\"></script>"
+ + ""
+ + "<script type=\"text/javascript\">"
+ + "$(document).ready(function(){"
+ + " $('textarea').autosize();"
+ + "});"
+ + "</script>"
+ + ""
+ + "<meta charset=utf-8 />"
+ + "<title>ASTERIX Demo</title>"
+ + "<style id=\"jsbin-css\">"
+ + "body {"
+ + " background: none repeat scroll 0 0 white;"
+ + " color: #222222;"
+ + " font-family: 'Bitter';"
+ + " font-size: 14px;"
+ + " line-height: 17px;"
+ + " width: 100%;"
+ + "}"
+ + ""
+ + ".content {"
+ + " margin-top: 70px;"
+ + "}"
+ + ""
+ + "label.query, label.result {"
+ + " font-size: 24px;"
+ + " padding-bottom: 10px;"
+ + " font-weight: bold;"
+ + "}"
+ + ""
+ + "div.host {"
+ + " float: left;"
+ + " margin: 0 100px 0 10px;"
+ + "}"
+ + ""
+ + "div.port {"
+ + "}"
+ + ""
+ + "div.left {"
+ + " float: left;"
+ + " width: 320px;"
+ + " padding: 0 20px 0 10px;"
+ + "}"
+ + ""
+ + "div.right {"
+ + "}"
+ + ""
+ + "button.btn {"
+ + " clear: both;"
+ + " float: left;"
+ + " margin: 20px 0 0 10px;;"
+ + "}"
+ + ""
+ + "textarea.query {"
+ + " -webkit-box-sizing: border-box;"
+ + " -moz-box-sizing: border-box;"
+ + " -ms-box-sizing: border-box;"
+ + " box-sizing: border-box;"
+ + " font-size: 16px;"
+ + " line-height: 20px;"
+ + " font-family: bitter, helvetica;"
+ + " width: 100%;"
+ + " padding: 10px;"
+ + " color: #999;"
+ + " resize: none;"
+ + " border: 10px solid #eee;"
+ + "}"
+ + ""
+ + "textarea.query:focus {"
+ + " outline: none;"
+ + " color: #333;"
+ + "}"
+ + ""
+ + "label {"
+ + " padding-top: 10px;"
+ + "}"
+ + ""
+ + "input[type=text] {"
+ + " height: 20px;"
+ + "}"
+ + ""
+ + ""
+ + "div.output label.heading {"
+ + " font-size: 24px;"
+ + " margin-top: 2px;"
+ + " padding-bottom: 10px;"
+ + " font-weight: bold;"
+ + "}"
+ + ""
+ + "div.output .message {"
+ + " -webkit-box-sizing: border-box;"
+ + " -moz-box-sizing: border-box;"
+ + " -ms-box-sizing: border-box;"
+ + " box-sizing: border-box;"
+ + " -webkit-border-radius: 4px 4px 4px 4px;"
+ + " -moz-border-radius: 4px 4px 4px 4px;"
+ + " -ms-border-radius: 4px 4px 4px 4px;"
+ + " border-radius: 4px 4px 4px 4px;"
+ + " color: #000;"
+ + " resize: none;"
+ + " border: 1px solid #eee;"
+ + "}"
+ + ""
+ + "div.error label.heading {"
+ + " color: #ff2020;"
+ + " font-size: 24px;"
+ + " margin-top: 2px;"
+ + " padding-bottom: 10px;"
+ + " font-weight: bold;"
+ + "}"
+ + ""
+ + "div.error .message {"
+ + " -webkit-box-sizing: border-box;"
+ + " -moz-box-sizing: border-box;"
+ + " -ms-box-sizing: border-box;"
+ + " box-sizing: border-box;"
+ + " border-color: rgba(82, 168, 236, 0.8);"
+ + " outline: 0;"
+ + " outline: thin dotted 9;"
+ + ""
+ + " -webkit-box-shadow: inset 0 1px 1px rgba(250, 0, 0, 0.075), 0 0 8px rgba(255, 0, 0, 0.8);"
+ + " -moz-box-shadow: inset 0 1px 1px rgba(250, 0, 0, 0.075), 0 0 8px rgba(255, 0, 0, 1.0);"
+ + " box-shadow: inset 0 1px 1px rgba(250, 0, 0, 0.075), 0 0 8px rgba(255, 0, 0, 1.0);"
+ + " color: #000;"
+ + " resize: none;"
+ + " border: 1px solid #eee;"
+ + " margin-top: 7px;"
+ + " padding: 20px 20px 20px 20px;"
+ + "}"
+ + ""
+ + ".footer {"
+ + " margin-top: 40px;"
+ + "}"
+ + ""
+ + ".footer .line {"
+ + " border-top: 1px solid #EEEEEE;"
+ + " bottom: 20px;"
+ + " height: 10px;"
+ + " left: 0;"
+ + " position: fixed;"
+ + " width: 100%;"
+ + "}"
+ + ""
+ + ".footer .content {"
+ + " background: none repeat scroll 0 0 #FFFFFF;"
+ + " bottom: 0;"
+ + " color: #666666;"
+ + " font-size: 12px;"
+ + " height: 25px;"
+ + " left: 0;"
+ + " padding-top: 5px;"
+ + " position: fixed;"
+ + " width: 100%;"
+ + "}"
+ + ""
+ + ".footer .content .left {"
+ + " padding-left: 20px;"
+ + " float: left;"
+ + "}"
+ + ""
+ + ".footer .content .right {"
+ + " padding-right: 20px;"
+ + " float: right;"
+ + "}</style></head>"
+ + "<body>"
+ + " <div class=\"navbar navbar-inverse navbar-fixed-top\">"
+ + " <div class=\"navbar-inner\">"
+ + " <div class=\"container\">"
+ + " <a class=\"btn btn-navbar\" data-toggle=\"collapse\" data-target=\".nav-collapse\">"
+ + " <span class=\"icon-bar\"></span>"
+ + " <span class=\"icon-bar\"></span>"
+ + " <span class=\"icon-bar\"></span>"
+ + " </a>"
+ + " <a class=\"brand\" href=\"#\">ASTERIX</a>"
+ + " <div class=\"nav-collapse collapse\">"
+ + " <ul class=\"nav\">"
+ + " <li><a href=\"#\">Open source</a></li>"
+ + " <li><a href=\"#about\">File issues</a></li>"
+ + " <li><a href=\"#contact\">Contact</a></li>"
+ + " </ul>"
+ + " </div><!--/.nav-collapse -->"
+ + " </div>"
+ + " </div>"
+ + " </div>"
+ + "";
+ private static final String HTML_FORM_CONTENT_TEMPLATE = " <div class=\"content\">"
+ + " <div class=\"container\">"
+ + " <div class=\"row-fluid\">"
+ + " <div class=\"span6\">"
+ + " <form class=\"form-horizontal\" method=\"post\">"
+ + " <div>"
+ + " <label class=\"query\">Query</label>"
+ + " <textarea rows=\"20\" name=\"query\" class=\"query\" value=\"%s\" placeholder=\"Type your AQL query ...\"></textarea>"
+ + " </div>"
+ + " <div>"
+ + " <div class=\"host\">"
+ + " <label>Host</label><input type=\"text\" name=\"hyracks-ip\" placeholder=\"IP Address or hostname\"/>"
+ + " </div>"
+ + " <div class=\"port\">"
+ + " <label>Port</label><input type=\"text\" name=\"hyracks-port\" placeholder=\"Port number\"/>"
+ + " </div>"
+ + " </div>"
+ + " <div>"
+ + " <div class=\"left\">"
+ + " <label class=\"checkbox\"><input type=\"checkbox\" name=\"print-expr-tree\" value=\"true\" /> Print parsed expressions</label>"
+ + " </div>"
+ + " <div class=\"right\">"
+ + " <label class=\"checkbox\"><input type=\"checkbox\" name=\"print-rewritten-expr-tree\" value=\"true\" /> Print rewritten expressions</label>"
+ + " </div>"
+ + " </div>"
+ + " <div>"
+ + " <div class=\"left\">"
+ + " <label class=\"checkbox\"><input type=\"checkbox\" name=\"print-logical-plan\" value=\"true\" /> Print logical plan</label>"
+ + " </div>"
+ + " <div class=\"right\">"
+ + " <label class=\"checkbox\"><input type=\"checkbox\" name=\"print-optimized-logical-plan\" value=\"true\" /> Print optimized logical plan</label>"
+ + " </div>"
+ + " </div>"
+ + " <div>"
+ + " <div class=\"left\">"
+ + " <label class=\"checkbox\"><input type=\"checkbox\" name=\"print-job\" value=\"true\" /> Print hyracks job</label>"
+ + " </div>"
+ + " <div class=\"right\">"
+ + " <label class=\"checkbox\"><input type=\"checkbox\" name=\"display-result\" value=\"true\" /> Display error/results</label>"
+ + " </div>"
+ + " </div>"
+ + " <button type=\"submit\" class=\"btn btn-danger\">Execute</button>"
+ + " </form>"
+ + " </div>";
+
+ private static final String HTML_EMPTY_OUTPUT_TEMPLATE = " <div class=\"span6\">"
+ + " <div class=\"output\">"
+ + " <label class=\"heading\">Output</label>"
+ + " <div class=\"message\">"
+ + " </div>"
+ + " </div>"
+ + " </div>";
+
+ private static final String HTML_OUTPUT_TEMPLATE = "<div class=\"span6\">"
+ + " <div class=\"output\">"
+ + " <label class=\"heading\">Output</label>"
+ + " <table class=\"table table-bordered table-striped\">"
+ + " %s"
+ + " </table>"
+ + " </div>"
+ + "</div>";
+
+ private static final String HTML_FOOTER_TEMPLATE = " </div>"
+ + " </div>"
+ + " </div>"
+ + " <div class=\"footer\">"
+ + " <section class=\"line\"><hr></section>"
+ + " <section class=\"content\">"
+ + " <section class=\"left\">"
+ + " Developed by ASTERIX group"
+ + " </section>"
+ + " <section class=\"right\">"
+ + " © Copyright 2012 University of California, Irvine"
+ + " </section>"
+ + " </section>"
+ + " </div>"
+ + "</body>"
+ + "</html>";
+
@Override
public void doPost(HttpServletRequest request, HttpServletResponse response) throws IOException {
String query = request.getParameter("query");
@@ -61,33 +337,12 @@
isSet(printOptimizedLogicalPlanParam), false, true, isSet(printJob));
MetadataManager.INSTANCE.init();
AqlTranslator aqlTranslator = new AqlTranslator(aqlStatements, out, sessionConfig, DisplayFormat.HTML);
- List<QueryResult> executionResults = null;
double duration = 0;
long startTime = System.currentTimeMillis();
- executionResults = aqlTranslator.compileAndExecute(hcc);
+ aqlTranslator.compileAndExecute(hcc, false);
long endTime = System.currentTimeMillis();
duration = (endTime - startTime) / 1000.00;
out.println("<PRE>Duration of all jobs: " + duration + "</PRE>");
-
- int queryCount = 1;
- out.println("<H1>Result:</H1>");
- out.println("<PRE>");
- for (QueryResult result : executionResults) {
- out.println("Query:" + queryCount++ + ":" + " " + result.getResultPath());
- }
- out.println("Duration: " + duration);
- out.println("</PRE>");
-
- queryCount = 1;
- if (isSet(strDisplayResult)) {
- out.println("<PRE>");
- for (QueryResult result : executionResults) {
- out.println("Query:" + queryCount++ + ":" + " " + result.getResultPath());
- displayFile(new File(result.getResultPath()), out);
- out.println();
- }
- out.println("</PRE>");
- }
} catch (ParseException pe) {
String message = pe.getMessage();
message = message.replace("<", "<");
@@ -110,21 +365,8 @@
public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
PrintWriter out = response.getWriter();
response.setContentType("text/html");
- final String form = "<form method=\"post\">"
- + "<center><textarea cols=\"80\" rows=\"25\" name=\"query\" ></textarea><br/>"
- + "IP Address: <input type = \"text\" name = \"hyracks-ip\" size=\"15\" maxlength=\"35\" value=\"localhost\" /><br/>"
- + "Port: <input type = \"text\" name = \"hyracks-port\" size=\"5\" maxlength=\"5\" value=\"1098\" /><br/>"
- + "<input type = \"checkbox\" name = \"print-expr-tree\" value=\"true\" />print parsed expressions<P>"
- + "<input type = \"checkbox\" name = \"print-rewritten-expr-tree\" value=\"true\" />print rewritten expressions<P>"
- + "<input type = \"checkbox\" name = \"print-logical-plan\" value=\"true\" checked/>print logical plan<P>"
- + "<input type = \"checkbox\" name = \"print-optimized-logical-plan\" value=\"true\" checked/>print optimized logical plan<P>"
- + "<input type = \"checkbox\" name = \"print-job\" value=\"true\" checked/>print Hyracks job<P>"
- + "<input type = \"checkbox\" name = \"display-result\" value=\"true\" checked/>display NFS file<P>"
- // +
- // "<input type = \"checkbox\" name = \"serialize-as-xml\" value=\"true\">serialize as XML<P>"
- // +
- // "<input type = \"checkbox\" name = \"show-tuples\" value=\"true\">show the entire tuples<P>"
- + "<input type=\"submit\"/>" + "</center>" + "</form>";
+ final String form = HTML_HEADER_TEMPLATE + String.format(HTML_FORM_CONTENT_TEMPLATE, "")
+ + HTML_EMPTY_OUTPUT_TEMPLATE + HTML_FOOTER_TEMPLATE;
out.println(form);
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
new file mode 100644
index 0000000..1ae3054
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.util.Arrays;
+import java.util.List;
+
+import javax.servlet.http.HttpServletRequest;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
+
+public class DDLAPIServlet extends RESTAPIServlet {
+ private static final long serialVersionUID = 1L;
+
+ protected String getQueryParameter(HttpServletRequest request) {
+ return request.getParameter("ddl");
+ }
+
+ protected List<Statement.Kind> getAllowedStatements() {
+ Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DATAVERSE_DROP, Kind.FUNCTION_DECL, Kind.NODEGROUP_DECL,
+ Kind.NODEGROUP_DROP, Kind.TYPE_DECL, Kind.TYPE_DROP, Kind.CREATE_INDEX, Kind.INDEX_DECL,
+ Kind.CREATE_DATAVERSE, Kind.INDEX_DROP, Kind.CREATE_FUNCTION, Kind.FUNCTION_DROP };
+ return Arrays.asList(statementsArray);
+ }
+
+ protected String getErrorMessage() {
+ return "Invalid statement: Non-DDL statement %s to the DDL API.";
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/HyracksProperties.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/HyracksProperties.java
new file mode 100644
index 0000000..c5b532a
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/HyracksProperties.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+public class HyracksProperties {
+ private final InputStream is;
+
+ private final Properties properties;
+
+ private static String HYRACKS_IP = "127.0.0.1";
+
+ private static int HYRACKS_PORT = 1098;
+
+ public HyracksProperties() throws IOException {
+ is = HyracksProperties.class.getClassLoader().getResourceAsStream("hyracks-deployment.properties");
+ properties = new Properties();
+ properties.load(is);
+ }
+
+ public String getHyracksIPAddress() {
+ String strIP = properties.getProperty("cc.ip");
+ if (strIP == null) {
+ strIP = HYRACKS_IP;
+ }
+ return strIP;
+ }
+
+ public int getHyracksPort() {
+ String strPort = properties.getProperty("cc.port");
+ int port = HYRACKS_PORT;
+ if (strPort != null) {
+ port = Integer.parseInt(strPort);
+ }
+ return port;
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryAPIServlet.java
new file mode 100644
index 0000000..a893bd6
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryAPIServlet.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.util.Arrays;
+import java.util.List;
+
+import javax.servlet.http.HttpServletRequest;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
+
+public class QueryAPIServlet extends RESTAPIServlet {
+ private static final long serialVersionUID = 1L;
+
+ protected String getQueryParameter(HttpServletRequest request) {
+ return request.getParameter("query");
+ }
+
+ protected List<Statement.Kind> getAllowedStatements() {
+ Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.QUERY, Kind.SET, Kind.WRITE, Kind.BEGIN_FEED,
+ Kind.CONTROL_FEED };
+ return Arrays.asList(statementsArray);
+ }
+
+ protected String getErrorMessage() {
+ return "Invalid statement: Non-query statement %s to the query API.";
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java
new file mode 100644
index 0000000..f4b091a
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.ByteBuffer;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.json.JSONArray;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.result.ResultReader;
+import edu.uci.ics.asterix.result.ResultUtils;
+import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public class QueryResultAPIServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ private static final String HYRACKS_CONNECTION_ATTR = "edu.uci.ics.asterix.HYRACKS_CONNECTION";
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+ String strHandle = request.getParameter("handle");
+ PrintWriter out = response.getWriter();
+ response.setContentType("text/html");
+ ServletContext context = getServletContext();
+ IHyracksClientConnection hcc;
+ try {
+ HyracksProperties hp = new HyracksProperties();
+ String strIP = hp.getHyracksIPAddress();
+ int port = hp.getHyracksPort();
+
+ synchronized (context) {
+ hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
+ if (hcc == null) {
+ hcc = new HyracksConnection(strIP, port);
+ context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+ }
+ }
+ JSONObject handleObj = new JSONObject(strHandle);
+ JSONArray handle = handleObj.getJSONArray("handle");
+ JobId jobId = new JobId(handle.getLong(0));
+ ResultSetId rsId = new ResultSetId(handle.getLong(1));
+ ByteBuffer buffer = ByteBuffer.allocate(ResultReader.FRAME_SIZE);
+ /* TODO(madhusudancs): We need to find a way to JSON serialize default format obtained from
+ * metadataProvider in the AQLTranslator and store it as part of the result handle.
+ */
+ ResultReader resultReader = new ResultReader(hcc, FormatUtils.getDefaultFormat());
+ resultReader.open(jobId, rsId);
+ buffer.clear();
+ JSONObject jsonResponse = new JSONObject();
+ JSONArray results = new JSONArray();
+ while (resultReader.read(buffer) > 0) {
+ results.put(ResultUtils.getJSONFromBuffer(buffer, resultReader.getFrameTupleAccessor(),
+ resultReader.getRecordDescriptor()));
+ }
+ jsonResponse.put("results", results);
+ out.write(jsonResponse.toString());
+
+ } catch (Exception e) {
+ out.println(e.getMessage());
+ e.printStackTrace(out);
+ }
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryStatusAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryStatusAPIServlet.java
new file mode 100644
index 0000000..964c4b0
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryStatusAPIServlet.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.json.JSONArray;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.result.ResultReader;
+import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public class QueryStatusAPIServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ private static final String HYRACKS_CONNECTION_ATTR = "edu.uci.ics.asterix.HYRACKS_CONNECTION";
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+ String strHandle = request.getParameter("handle");
+ PrintWriter out = response.getWriter();
+ response.setContentType("text/html");
+ ServletContext context = getServletContext();
+ IHyracksClientConnection hcc;
+ try {
+ HyracksProperties hp = new HyracksProperties();
+ String strIP = hp.getHyracksIPAddress();
+ int port = hp.getHyracksPort();
+
+ synchronized (context) {
+ hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
+ if (hcc == null) {
+ hcc = new HyracksConnection(strIP, port);
+ context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+ }
+ }
+ JSONObject handleObj = new JSONObject(strHandle);
+ JSONArray handle = handleObj.getJSONArray("handle");
+ JobId jobId = new JobId(handle.getLong(0));
+ ResultSetId rsId = new ResultSetId(handle.getLong(1));
+
+ /* TODO(madhusudancs): We need to find a way to JSON serialize default format obtained from
+ * metadataProvider in the AQLTranslator and store it as part of the result handle.
+ */
+ ResultReader resultReader = new ResultReader(hcc, FormatUtils.getDefaultFormat());
+ resultReader.open(jobId, rsId);
+
+ JSONObject jsonResponse = new JSONObject();
+ String status;
+ switch (resultReader.getStatus()) {
+ case RUNNING:
+ status = "RUNNING";
+ break;
+ case FAILED:
+ status = "ERROR";
+ break;
+ case SUCCESS:
+ status = "SUCCESS";
+ break;
+ default:
+ status = "ERROR";
+ break;
+ }
+ jsonResponse.put("status", status);
+ out.write(jsonResponse.toString());
+
+ } catch (Exception e) {
+ out.println(e.getMessage());
+ e.printStackTrace(out);
+ }
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
new file mode 100644
index 0000000..abf89f0
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.util.List;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
+import edu.uci.ics.asterix.api.common.SessionConfig;
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
+import edu.uci.ics.asterix.aql.parser.AQLParser;
+import edu.uci.ics.asterix.aql.parser.ParseException;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.result.ResultUtils;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+
+abstract class RESTAPIServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ private static final String HYRACKS_CONNECTION_ATTR = "edu.uci.ics.asterix.HYRACKS_CONNECTION";
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+ PrintWriter out = response.getWriter();
+
+ String query = getQueryParameter(request);
+ boolean asyncResults = isAsync(request);
+
+ response.setContentType("application/json");
+ ServletContext context = getServletContext();
+ IHyracksClientConnection hcc;
+
+ try {
+ HyracksProperties hp = new HyracksProperties();
+ String strIP = hp.getHyracksIPAddress();
+ int port = hp.getHyracksPort();
+
+ synchronized (context) {
+ hcc = (IHyracksClientConnection) context.getAttribute(HYRACKS_CONNECTION_ATTR);
+ if (hcc == null) {
+ hcc = new HyracksConnection(strIP, port);
+ context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+ }
+ }
+ AQLParser parser = new AQLParser(new StringReader(query));
+ List<Statement> aqlStatements = parser.Statement();
+ if (checkForbiddenStatements(aqlStatements, out)) {
+ return;
+ }
+ SessionConfig sessionConfig = new SessionConfig(port, true, false, false, false, false, false, true, false);
+
+ MetadataManager.INSTANCE.init();
+
+ AqlTranslator aqlTranslator = new AqlTranslator(aqlStatements, out, sessionConfig, DisplayFormat.JSON);
+
+ aqlTranslator.compileAndExecute(hcc, asyncResults);
+
+ } catch (ParseException pe) {
+ StringBuilder errorMessage = new StringBuilder();
+ String message = pe.getMessage();
+ message = message.replace("<", "<");
+ message = message.replace(">", ">");
+ errorMessage.append("SyntaxError:" + message + "\n");
+ int pos = message.indexOf("line");
+ if (pos > 0) {
+ int columnPos = message.indexOf(",", pos + 1 + "line".length());
+ int lineNo = Integer.parseInt(message.substring(pos + "line".length() + 1, columnPos));
+ String line = query.split("\n")[lineNo - 1];
+ errorMessage.append("==> " + line + "\n");
+ }
+ JSONObject errorResp = ResultUtils.getErrorResponse(2, errorMessage.toString());
+ out.write(errorResp.toString());
+ } catch (Exception e) {
+ StringBuilder errorMessage = new StringBuilder();
+ errorMessage.append(e.getMessage());
+ JSONObject errorResp = ResultUtils.getErrorResponse(99, errorMessage.toString());
+ out.write(errorResp.toString());
+ }
+ }
+
+ private boolean checkForbiddenStatements(List<Statement> aqlStatements, PrintWriter out) {
+ for (Statement st : aqlStatements) {
+ if (!getAllowedStatements().contains(st.getKind())) {
+ JSONObject errorResp = ResultUtils.getErrorResponse(1, String.format(getErrorMessage(), st.getKind()));
+ out.write(errorResp.toString());
+ return true;
+ }
+ }
+ return false;
+ }
+
+ protected boolean isAsync(HttpServletRequest request) {
+ String mode = request.getParameter("mode");
+ boolean asyncResults = false;
+ if (mode != null && mode.equals("asynchronous")) {
+ asyncResults = true;
+ }
+ return asyncResults;
+ }
+
+ protected abstract String getQueryParameter(HttpServletRequest request);
+
+ protected abstract List<Kind> getAllowedStatements();
+
+ protected abstract String getErrorMessage();
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
new file mode 100644
index 0000000..7336876
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2011 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.http.servlet;
+
+import java.util.Arrays;
+import java.util.List;
+
+import javax.servlet.http.HttpServletRequest;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
+
+public class UpdateAPIServlet extends RESTAPIServlet {
+ private static final long serialVersionUID = 1L;
+
+ protected String getQueryParameter(HttpServletRequest request) {
+ return request.getParameter("statements");
+ }
+
+ protected List<Statement.Kind> getAllowedStatements() {
+ Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DATASET_DECL, Kind.DATASET_DROP, Kind.DELETE, Kind.INSERT,
+ Kind.UPDATE, Kind.DML_CMD_LIST, Kind.LOAD_FROM_FILE, Kind.WRITE_FROM_QUERY_RESULT };
+ return Arrays.asList(statementsArray);
+ }
+
+ protected String getErrorMessage() {
+ return "Invalid statement: Non-Update statement %s to the Update API.";
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
index 7e72d94..fb09299 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
@@ -52,7 +52,7 @@
int ch;
StringBuilder builder = new StringBuilder();
while ((ch = queryText.read()) != -1) {
- builder.append((char)ch);
+ builder.append((char) ch);
}
AQLParser parser = new AQLParser(builder.toString());
List<Statement> aqlStatements;
@@ -68,7 +68,7 @@
generateBinaryRuntime, printJob);
AqlTranslator aqlTranslator = new AqlTranslator(aqlStatements, writer, pc, DisplayFormat.TEXT);
- aqlTranslator.compileAndExecute(hcc);
+ aqlTranslator.compileAndExecute(hcc, false);
writer.flush();
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 66239e2..ccc72c1 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -16,6 +16,7 @@
import java.io.File;
import java.io.PrintWriter;
+import java.nio.ByteBuffer;
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.HashMap;
@@ -23,12 +24,15 @@
import java.util.Map;
import org.json.JSONException;
+import org.json.JSONArray;
+import org.json.JSONObject;
import edu.uci.ics.asterix.api.common.APIFramework;
import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
import edu.uci.ics.asterix.api.common.Job;
import edu.uci.ics.asterix.api.common.SessionConfig;
import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
@@ -81,6 +85,8 @@
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.types.TypeSignature;
+import edu.uci.ics.asterix.result.ResultUtils;
+import edu.uci.ics.asterix.result.ResultReader;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
@@ -101,6 +107,7 @@
import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
import edu.uci.ics.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -140,15 +147,15 @@
/**
* Compiles and submits for execution a list of AQL statements.
- *
* @param hcc
- * AHyracks client connection that is used to submit a jobspec to
- * Hyracks.
- * @return A List<QueryResult> containing a QueryResult instance
- * corresponding to each submitted query.
+ * A Hyracks client connection that is used to submit a jobspec to Hyracks.
+ * @param asyncResults
+ * True if the results should be read asynchronously or false if we should wait for results to be read.
+ * @return A List<QueryResult> containing a QueryResult instance corresponding to each submitted query.
* @throws Exception
*/
- public List<QueryResult> compileAndExecute(IHyracksClientConnection hcc) throws Exception {
+ public List<QueryResult> compileAndExecute(IHyracksClientConnection hcc, boolean asyncResults) throws Exception {
+ int resultSetIdCounter = 0;
List<QueryResult> executionResult = new ArrayList<QueryResult>();
FileSplit outputFile = null;
IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
@@ -252,6 +259,7 @@
}
case QUERY: {
+ metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
executionResult.add(handleQuery(metadataProvider, (Query) stmt, hcc, jobsToExecute));
break;
}
@@ -275,7 +283,30 @@
// Following jobs are run under a separate transaction, that is
// committed/aborted by the JobEventListener
for (JobSpecification jobspec : jobsToExecute) {
- runJob(hcc, jobspec);
+ JobId jobId = runJob(hcc, jobspec);
+ if (stmt.getKind() == Kind.QUERY) {
+ JSONObject response = new JSONObject();
+
+ if (asyncResults) {
+ JSONArray handle = new JSONArray();
+ handle.put(jobId.getId());
+ handle.put(metadataProvider.getResultSetId().getId());
+ response.put("handle", handle);
+ } else {
+ ByteBuffer buffer = ByteBuffer.allocate(ResultReader.FRAME_SIZE);
+ ResultReader resultReader = new ResultReader(hcc, metadataProvider.getFormat());
+ resultReader.open(jobId, metadataProvider.getResultSetId());
+ buffer.clear();
+ JSONArray results = new JSONArray();
+ while (resultReader.read(buffer) > 0) {
+ results.put(ResultUtils.getJSONFromBuffer(buffer, resultReader.getFrameTupleAccessor(),
+ resultReader.getRecordDescriptor()));
+ }
+ response.put("results", results);
+ }
+ out.write(response.toString());
+ }
+ hcc.waitForCompletion(jobId);
}
}
return executionResult;
@@ -393,7 +424,8 @@
if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
dataverseName);
- runJob(hcc, DatasetOperations.createDatasetJobSpec(dataverse, datasetName, metadataProvider));
+ JobId jobId = runJob(hcc, DatasetOperations.createDatasetJobSpec(dataverse, datasetName, metadataProvider));
+ hcc.waitForCompletion(jobId);
}
}
@@ -431,7 +463,8 @@
index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(), index.getIndexType());
JobSpecification loadIndexJobSpec = IndexOperations
.buildSecondaryIndexLoadingJobSpec(cis, metadataProvider);
- runJob(hcc, loadIndexJobSpec);
+ JobId jobId = runJob(hcc, loadIndexJobSpec);
+ hcc.waitForCompletion(jobId);
}
}
@@ -664,9 +697,9 @@
CompiledWriteFromQueryResultStatement clfrqs = new CompiledWriteFromQueryResultStatement(dataverseName, st1
.getDatasetName().getValue(), st1.getQuery(), st1.getVarCounter());
- Pair<JobSpecification, FileSplit> compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
- if (compiled.first != null) {
- jobsToExecute.add(compiled.first);
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+ if (compiled != null) {
+ jobsToExecute.add(compiled);
}
}
@@ -678,9 +711,10 @@
: activeDefaultDataverse.getDataverseName() : stmtInsert.getDataverseName().getValue();
CompiledInsertStatement clfrqs = new CompiledInsertStatement(dataverseName, stmtInsert.getDatasetName()
.getValue(), stmtInsert.getQuery(), stmtInsert.getVarCounter());
- Pair<JobSpecification, FileSplit> compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
- if (compiled.first != null) {
- jobsToExecute.add(compiled.first);
+
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+ if (compiled != null) {
+ jobsToExecute.add(compiled);
}
}
@@ -693,13 +727,13 @@
CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getDieClause(),
stmtDelete.getVarCounter(), metadataProvider);
- Pair<JobSpecification, FileSplit> compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
- if (compiled.first != null) {
- jobsToExecute.add(compiled.first);
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+ if (compiled != null) {
+ jobsToExecute.add(compiled);
}
}
- private Pair<JobSpecification, FileSplit> rewriteCompileQuery(AqlMetadataProvider metadataProvider, Query query,
+ private JobSpecification rewriteCompileQuery(AqlMetadataProvider metadataProvider, Query query,
ICompiledDmlStatement stmt) throws AsterixException, RemoteException, AlgebricksException, JSONException,
ACIDException {
@@ -715,9 +749,7 @@
JobSpecification spec = APIFramework.compileQuery(declaredFunctions, metadataProvider, query,
reWrittenQuery.second, stmt == null ? null : stmt.getDatasetName(), sessionConfig, out, pdf, stmt);
- Pair<JobSpecification, FileSplit> compiled = new Pair<JobSpecification, FileSplit>(spec,
- metadataProvider.getOutputFile());
- return compiled;
+ return spec;
}
@@ -741,9 +773,10 @@
}
bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
cbfs.setQuery(bfs.getQuery());
- Pair<JobSpecification, FileSplit> compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
- if (compiled.first != null) {
- jobsToExecute.add(compiled.first);
+
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
+ if (compiled != null) {
+ jobsToExecute.add(compiled);
}
}
@@ -759,12 +792,13 @@
private QueryResult handleQuery(AqlMetadataProvider metadataProvider, Query query, IHyracksClientConnection hcc,
List<JobSpecification> jobsToExecute) throws Exception {
- Pair<JobSpecification, FileSplit> compiled = rewriteCompileQuery(metadataProvider, query, null);
- if (compiled.first != null) {
- GlobalConfig.ASTERIX_LOGGER.info(compiled.first.toJSON().toString(1));
- jobsToExecute.add(compiled.first);
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, query, null);
+ if (compiled != null) {
+ GlobalConfig.ASTERIX_LOGGER.info(compiled.toJSON().toString(1));
+ jobsToExecute.add(compiled);
}
- return new QueryResult(query, compiled.second.getLocalFile().getFile().getAbsolutePath());
+
+ return new QueryResult(query, metadataProvider.getResultSetId());
}
private void runCreateIndexJob(IHyracksClientConnection hcc, CreateIndexStatement stmtCreateIndex,
@@ -783,7 +817,8 @@
throw new AsterixException("Failed to create job spec for creating index '"
+ stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
}
- runJob(hcc, spec);
+ JobId jobId = runJob(hcc, spec);
+ hcc.waitForCompletion(jobId);
}
private void handleCreateNodeGroupStatement(AqlMetadataProvider metadataProvider, Statement stmt,
@@ -806,14 +841,16 @@
}
}
- private void runJob(IHyracksClientConnection hcc, JobSpecification spec) throws Exception {
- executeJobArray(hcc, new Job[] { new Job(spec) }, out, pdf);
+ private JobId runJob(IHyracksClientConnection hcc, JobSpecification spec) throws Exception {
+ JobId[] jobIds = executeJobArray(hcc, new Job[] { new Job(spec) }, out, pdf);
+ return jobIds[0];
}
private void compileIndexDropStatement(IHyracksClientConnection hcc, String dataverseName, String datasetName,
String indexName, AqlMetadataProvider metadataProvider) throws Exception {
CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
- runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+ JobId jobId = runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+ hcc.waitForCompletion(jobId);
MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName,
indexName);
}
@@ -825,20 +862,24 @@
Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
JobSpecification[] jobSpecs = DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider);
- for (JobSpecification spec : jobSpecs)
- runJob(hcc, spec);
+ for (JobSpecification spec : jobSpecs) {
+ JobId jobId = runJob(hcc, spec);
+ hcc.waitForCompletion(jobId);
+ }
}
MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
}
- public void executeJobArray(IHyracksClientConnection hcc, Job[] jobs, PrintWriter out, DisplayFormat pdf)
+ public JobId[] executeJobArray(IHyracksClientConnection hcc, Job[] jobs, PrintWriter out, DisplayFormat pdf)
throws Exception {
+ JobId[] startedJobIds = new JobId[jobs.length];
for (int i = 0; i < jobs.length; i++) {
JobSpecification spec = jobs[i].getJobSpec();
spec.setMaxReattempts(0);
JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, spec);
- hcc.waitForCompletion(jobId);
+ startedJobIds[i] = jobId;
}
+ return startedJobIds;
}
private static IDataFormat getDataFormat(MetadataTransactionContext mdTxnCtx, String dataverseName)
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/QueryResult.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/QueryResult.java
index 0b4a2cf..8d6b35e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/QueryResult.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/QueryResult.java
@@ -16,23 +16,35 @@
import edu.uci.ics.asterix.aql.base.Statement;
import edu.uci.ics.asterix.aql.expression.Query;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
public class QueryResult {
private final Query query;
- private final String resultPath;
- public QueryResult(Query statement, String resultPath) {
+ private final ResultSetId resultSetId;
+
+ private JobId jobId;
+
+ public QueryResult(Query statement, ResultSetId resultSetId) {
this.query = statement;
- this.resultPath = resultPath;
+ this.resultSetId = resultSetId;
+ }
+
+ public void setJobId(JobId jobId) {
+ this.jobId = jobId;
+ }
+
+ public JobId getJobId() {
+ return jobId;
}
public Statement getStatement() {
return query;
}
- public String getResultPath() {
- return resultPath;
+ public ResultSetId getResultSetId() {
+ return resultSetId;
}
-
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java
index 9b29427..0c25e34 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCBootstrapImpl.java
@@ -23,6 +23,11 @@
import edu.uci.ics.asterix.api.http.servlet.APIServlet;
import edu.uci.ics.asterix.common.api.AsterixAppContextInfoImpl;
+import edu.uci.ics.asterix.api.http.servlet.DDLAPIServlet;
+import edu.uci.ics.asterix.api.http.servlet.QueryAPIServlet;
+import edu.uci.ics.asterix.api.http.servlet.QueryResultAPIServlet;
+import edu.uci.ics.asterix.api.http.servlet.QueryStatusAPIServlet;
+import edu.uci.ics.asterix.api.http.servlet.UpdateAPIServlet;
import edu.uci.ics.asterix.common.config.GlobalConfig;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
@@ -40,7 +45,10 @@
private static final int DEFAULT_WEB_SERVER_PORT = 19001;
+ private static final int DEFAULT_JSON_API_SERVER_PORT = 19101;
+
private Server webServer;
+ private Server jsonAPIServer;
private static IAsterixStateProxy proxy;
private ICCApplicationContext appCtx;
@@ -59,6 +67,10 @@
setupWebServer();
webServer.start();
+ // Setup and start the web interface
+ setupJSONAPIServer();
+ jsonAPIServer.start();
+
AsterixAppContextInfoImpl.initialize(appCtx);
}
@@ -90,4 +102,22 @@
webServer.setHandler(context);
context.addServlet(new ServletHolder(new APIServlet()), "/*");
}
-}
\ No newline at end of file
+
+ private void setupJSONAPIServer() throws Exception {
+ String portStr = System.getProperty(GlobalConfig.JSON_API_SERVER_PORT_PROPERTY);
+ int port = DEFAULT_JSON_API_SERVER_PORT;
+ if (portStr != null) {
+ port = Integer.parseInt(portStr);
+ }
+ jsonAPIServer = new Server(port);
+
+ ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+ context.setContextPath("/");
+ jsonAPIServer.setHandler(context);
+ context.addServlet(new ServletHolder(new QueryAPIServlet()), "/query");
+ context.addServlet(new ServletHolder(new QueryStatusAPIServlet()), "/query/status");
+ context.addServlet(new ServletHolder(new QueryResultAPIServlet()), "/query/result");
+ context.addServlet(new ServletHolder(new UpdateAPIServlet()), "/update");
+ context.addServlet(new ServletHolder(new DDLAPIServlet()), "/ddl");
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
new file mode 100644
index 0000000..8137e02
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2010 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.result;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.formats.base.IDataFormat;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+import edu.uci.ics.hyracks.client.dataset.DatasetClientContext;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class ResultReader {
+ private final DatasetClientContext datasetClientCtx;
+
+ private final IHyracksDataset hyracksDataset;
+
+ private final IDataFormat format;
+
+ private RecordDescriptor recordDescriptor;
+
+ private FrameTupleAccessor frameTupleAccessor;
+
+ // Number of parallel result reader buffers
+ private static final int NUM_READERS = 1;
+
+ // 32K buffer size;
+ public static final int FRAME_SIZE = 32768;
+
+ public ResultReader(IHyracksClientConnection hcc, IDataFormat format) throws Exception {
+ this.format = format;
+
+ datasetClientCtx = new DatasetClientContext(FRAME_SIZE);
+ hyracksDataset = new HyracksDataset(hcc, datasetClientCtx, NUM_READERS);
+ }
+
+ public void open(JobId jobId, ResultSetId resultSetId) throws IOException, ClassNotFoundException {
+ hyracksDataset.open(jobId, resultSetId);
+ byte[] serializedRecordDescriptor = hyracksDataset.getSerializedRecordDescriptor();
+
+ recordDescriptor = (RecordDescriptor) JavaSerializationUtils.deserialize(serializedRecordDescriptor, format
+ .getSerdeProvider().getClass().getClassLoader());
+
+ frameTupleAccessor = new FrameTupleAccessor(datasetClientCtx.getFrameSize(), recordDescriptor);
+ }
+
+ public Status getStatus() {
+ return hyracksDataset.getResultStatus();
+ }
+
+ public int read(ByteBuffer buffer) throws HyracksDataException {
+ return hyracksDataset.read(buffer);
+ }
+
+ public FrameTupleAccessor getFrameTupleAccessor() {
+ return frameTupleAccessor;
+ }
+
+ public RecordDescriptor getRecordDescriptor() {
+ return recordDescriptor;
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
new file mode 100644
index 0000000..adb8573
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.result;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultUtils {
+ public static JSONArray getJSONFromBuffer(ByteBuffer buffer, FrameTupleAccessor fta,
+ RecordDescriptor recordDescriptor) throws HyracksDataException {
+ JSONArray resultRecords = new JSONArray();
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream di = new DataInputStream(bbis);
+
+ try {
+ fta.reset(buffer);
+ for (int tIndex = 0; tIndex < fta.getTupleCount(); tIndex++) {
+ int start = fta.getTupleStartOffset(tIndex) + fta.getFieldSlotsLength();
+ bbis.setByteBuffer(buffer, start);
+ Object[] record = new Object[recordDescriptor.getFieldCount()];
+ JSONArray resultRecord = new JSONArray();
+ for (int i = 0; i < record.length; ++i) {
+ IAObject instance = (IAObject) recordDescriptor.getFields()[i].deserialize(di);
+ resultRecord.put(instance.toJSON());
+ }
+ resultRecords.put(resultRecord);
+ }
+ } catch (JSONException e) {
+ throw new HyracksDataException(e);
+ }
+ return resultRecords;
+ }
+
+ public static JSONObject getErrorResponse(int errorCode, String errorMessage) {
+ JSONObject errorResp = new JSONObject();
+ JSONArray errorArray = new JSONArray();
+ errorArray.put(errorCode);
+ errorArray.put(errorMessage);
+ try {
+ errorResp.put("error-code", errorArray);
+ } catch (JSONException e) {
+ // TODO(madhusudancs): Figure out what to do when JSONException occurs while building the results.
+ }
+ return errorResp;
+ }
+}
diff --git a/asterix-app/src/main/resources/hyracks-deployment.properties b/asterix-app/src/main/resources/hyracks-deployment.properties
index a8a943e..a333d38 100644
--- a/asterix-app/src/main/resources/hyracks-deployment.properties
+++ b/asterix-app/src/main/resources/hyracks-deployment.properties
@@ -1,2 +1,4 @@
cc.bootstrap.class=edu.uci.ics.asterix.hyracks.bootstrap.CCBootstrapImpl
-nc.bootstrap.class=edu.uci.ics.asterix.hyracks.bootstrap.NCBootstrapImpl
\ No newline at end of file
+nc.bootstrap.class=edu.uci.ics.asterix.hyracks.bootstrap.NCBootstrapImpl
+cc.ip=127.0.0.1
+cc.port=1098
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
index bb39f90..91ea627 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
@@ -4,9 +4,9 @@
public class GlobalConfig {
public static final boolean DEBUG = true;
-
+
public static final String ASTERIX_LOGGER_NAME = "edu.uci.ics.asterix";
-
+
public static final Logger ASTERIX_LOGGER = Logger.getLogger(ASTERIX_LOGGER_NAME);
public static final String ASTERIX_LOGFILE_PATTERN = "%t/asterix.log";
@@ -19,6 +19,8 @@
public static final String WEB_SERVER_PORT_PROPERTY = "AsterixWebServerPort";
+ public static final String JSON_API_SERVER_PORT_PROPERTY = "AsterixJSONAPIServerPort";
+
public static final String BUFFER_CACHE_PAGE_SIZE_PROPERTY = "BufferCachePageSize";
public static final String BUFFER_CACHE_NUM_PAGES_PROPERTY = "BufferCacheNumPages";
@@ -32,6 +34,6 @@
public static final String FRAME_SIZE_PROPERTY = "FrameSize";
public static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
-
+
public static int DEFAULT_INPUT_DATA_COLUMN = 0;
}
diff --git a/asterix-dist/src/main/scripts/startasterix.sh b/asterix-dist/src/main/scripts/startasterix.sh
index b84834f..f9d6e6c 100644
--- a/asterix-dist/src/main/scripts/startasterix.sh
+++ b/asterix-dist/src/main/scripts/startasterix.sh
@@ -67,8 +67,8 @@
# start the node controllers
for ((i=1;i<=$numnc;i++)); do
echo "node controller (nc$i) starting..."
- sh $HYRACKS_SERVER_BIN/hyracksnc -cc-host localhost -cluster-net-ip-address 127.0.0.1 -data-ip-address 127.0.0.1 -node-id "nc$i" \
- &> $LOG_DIR/nc$i.log &
+ sh $HYRACKS_SERVER_BIN/hyracksnc -cc-host localhost -cluster-net-ip-address 127.0.0.1 -data-ip-address 127.0.0.1 \
+ -result-ip-address 127.0.0.1 -node-id "nc$i" &> $LOG_DIR/nc$i.log &
# avoid socket error
sleep .5
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 321a73c..9183d6f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -91,6 +91,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -98,6 +99,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
@@ -120,6 +122,7 @@
private Map<String, String> config;
private IAWriterFactory writerFactory;
private FileSplit outputFile;
+ private ResultSetId resultSetId;
private long jobTxnId;
private final Dataverse defaultDataverse;
@@ -184,6 +187,14 @@
this.outputFile = outputFile;
}
+ public ResultSetId getResultSetId() {
+ return resultSetId;
+ }
+
+ public void setResultSetId(ResultSetId resultSetId) {
+ this.resultSetId = resultSetId;
+ }
+
@Override
public AqlDataSource findDataSource(AqlSourceId id) throws AlgebricksException {
AqlSourceId aqlId = (AqlSourceId) id;
@@ -542,6 +553,25 @@
}
@Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
+ RecordDescriptor recordDescriptor, boolean ordered, JobSpecification spec) throws AlgebricksException {
+ ResultSetDataSink rsds = (ResultSetDataSink) sink;
+ ResultSetSinkId rssId = (ResultSetSinkId) rsds.getId();
+ ResultSetId rsId = rssId.getResultSetId();
+ String nodeName = rssId.getResultNodeName();
+
+ ResultWriterOperatorDescriptor resultWriter = null;
+ try {
+ resultWriter = new ResultWriterOperatorDescriptor(spec, rsId, ordered, recordDescriptor);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+
+ AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeName });
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(resultWriter, apc);
+ }
+
+ @Override
public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
throws AlgebricksException {
AqlDataSource ads = findDataSource(dataSourceId);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
new file mode 100644
index 0000000..53513d2
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 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.metadata.declared;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+
+public class ResultSetDataSink implements IDataSink {
+
+ private ResultSetSinkId id;
+ private Object[] schemaTypes;
+
+ public ResultSetDataSink(ResultSetSinkId id, Object[] schemaTypes) {
+ this.id = id;
+ this.schemaTypes = schemaTypes;
+ }
+
+ @Override
+ public ResultSetSinkId getId() {
+ return id;
+ }
+
+ @Override
+ public Object[] getSchemaTypes() {
+ return schemaTypes;
+ }
+
+ @Override
+ public IPartitioningProperty getPartitioningProperty() {
+ return IPartitioningProperty.UNPARTITIONED;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
new file mode 100644
index 0000000..fc5152f
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 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.metadata.declared;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+
+public class ResultSetSinkId {
+
+ private final ResultSetId resultSetId;
+
+ private final String resultNodeName;
+
+ public ResultSetSinkId(ResultSetId resultSetId, String resultNodeName) {
+ this.resultSetId = resultSetId;
+ this.resultNodeName = resultNodeName;
+ }
+
+ @Override
+ public String toString() {
+ return "ResultSetId: " + resultSetId + "@" + resultNodeName;
+ }
+
+ public ResultSetId getResultSetId() {
+ return resultSetId;
+ }
+
+ public String getResultNodeName() {
+ return resultNodeName;
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABinary.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABinary.java
index db9ac03..47e23fc 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABinary.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABinary.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -74,4 +77,13 @@
public String toString() {
return "ABinary";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ABinary", bytes);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABitArray.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABitArray.java
index 4178b5a..a27b05e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABitArray.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABitArray.java
@@ -14,6 +14,10 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -129,4 +133,17 @@
sb.append(" ]");
return sb.toString();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONArray bitArray = new JSONArray();
+ for (int i = 0; i < intArray.length; i++) {
+ bitArray.put(intArray[i]);
+ }
+ json.put("ABitArray", bitArray);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABoolean.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABoolean.java
index 11b223a..85c35b6 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABoolean.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ABoolean.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -71,4 +74,13 @@
public int hash() {
return bVal.hashCode();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ABoolean", bVal);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ACircle.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ACircle.java
index 767f172..c7d946c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ACircle.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ACircle.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -73,4 +76,16 @@
public String toString() {
return "ACircle: { center: " + center + ", radius: " + radius + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONObject circle = new JSONObject();
+ circle.put("center", center);
+ circle.put("radius", radius);
+ json.put("ACircle", circle);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java
index 097b736..0ef170e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -82,4 +85,13 @@
public int getChrononTimeInDays() {
return chrononTimeInDay;
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ADate", chrononTimeInDay);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java
index e72fd8f..081fa63 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -113,4 +116,12 @@
return chrononTime;
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ADateTime", chrononTime);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADouble.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADouble.java
index 816b8be..582e192 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADouble.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADouble.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -70,4 +73,12 @@
return hashCode();
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ADouble", value);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADuration.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADuration.java
index 13f4a05..113751f 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADuration.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADuration.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -109,4 +112,15 @@
return sbder.toString();
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONObject duration = new JSONObject();
+ duration.put("months", chrononInMonth);
+ duration.put("milliseconds", chrononInMillisecond);
+ json.put("ADuration", duration);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AFloat.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AFloat.java
index a66e198..9804eb3 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AFloat.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AFloat.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -69,4 +72,13 @@
public String toString() {
return "AFloat: {" + value + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("AFloat", value);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt16.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt16.java
index bd27f20..2be714d 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt16.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt16.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -59,4 +62,13 @@
public String toString() {
return "AInt16: {" + value + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("AInt16", value);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt32.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt32.java
index b656a57..640bc0e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt32.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt32.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -84,4 +87,13 @@
public int hash() {
return hashCode();
}
-}
\ No newline at end of file
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("AInt32", value);
+
+ return json;
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
index 193a4af..769d492 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -58,4 +61,13 @@
public String toString() {
return "AInt64: {" + value + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("AInt64", value);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt8.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt8.java
index 87e5472..90135a3 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt8.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt8.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -58,4 +61,13 @@
public String toString() {
return "AInt8: {" + value + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("AInt8", value);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ALine.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ALine.java
index 0285825..dc420ed 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ALine.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ALine.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -66,4 +69,16 @@
public String toString() {
return "ALine: { p1: " + p1 + ", p2: " + p2 + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONObject line = new JSONObject();
+ line.put("p1", p1);
+ line.put("p2", p2);
+ json.put("ALine", line);
+
+ return json;
+ }
}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ANull.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ANull.java
index be47b41..baaa00a 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ANull.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ANull.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -52,4 +55,13 @@
public String toString() {
return "null";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ANull", "null");
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AOrderedList.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AOrderedList.java
index 4f5effd..36a92f3 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AOrderedList.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AOrderedList.java
@@ -16,6 +16,10 @@
import java.util.ArrayList;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.AOrderedListType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -107,4 +111,17 @@
sb.append(" ]");
return sb.toString();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONArray list = new JSONArray();
+ for (IAObject v : values) {
+ list.put(v.toJSON());
+ }
+ json.put("AOrderedList", list);
+
+ return json;
+ }
}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint.java
index 8e261a2..8bb44eb 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -66,4 +69,16 @@
public String toString() {
return "APoint: { x: " + x + ", y: " + y + " }";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONObject point = new JSONObject();
+ point.put("x", x);
+ point.put("y", y);
+ json.put("APoint", point);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint3D.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint3D.java
index 29ef6cd..d15139a 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint3D.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APoint3D.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -72,4 +75,17 @@
public String toString() {
return "APoint3D: { x: " + x + ", y: " + y + ", z: " + z + " }";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONObject point = new JSONObject();
+ point.put("x", x);
+ point.put("y", y);
+ point.put("z", z);
+ json.put("APoint3D", point);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APolygon.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APolygon.java
index 29e3727..ed21378 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APolygon.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/APolygon.java
@@ -14,6 +14,10 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -85,4 +89,17 @@
sb.append(" ]");
return sb.toString();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONArray polygon = new JSONArray();
+ for (int i = 0; i < points.length; i++) {
+ polygon.put(points[i].toJSON());
+ }
+ json.put("APolygon", polygon);
+
+ return json;
+ }
}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARecord.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARecord.java
index 0d1f578..0035289 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARecord.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARecord.java
@@ -14,6 +14,10 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.visitors.IOMVisitor;
@@ -87,4 +91,19 @@
sb.append(" }");
return sb.toString();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONArray record = new JSONArray();
+ for (int i = 0; i < fields.length; i++) {
+ JSONObject item = new JSONObject();
+ item.put(type.getFieldNames()[i], fields[i]);
+ record.put(item);
+ }
+ json.put("ARecord", record);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARectangle.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARectangle.java
index 51f06bf..d82d351 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARectangle.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ARectangle.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -66,4 +69,16 @@
public String toString() {
return "ARectangle: { p1: " + p1 + ", p2: " + p2 + "}";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONObject rectangle = new JSONObject();
+ rectangle.put("p1", p1);
+ rectangle.put("p2", p2);
+ json.put("ARectangle", rectangle);
+
+ return json;
+ }
}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AString.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AString.java
index 0f7c9c2..b1f2550 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AString.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AString.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -67,4 +70,13 @@
public int hash() {
return hashCode();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("AString", value);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java
index 7b66cf9..b97ed51 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.asterix.om.base;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -98,4 +101,12 @@
return chrononTime;
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ATime", chrononTime);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUnorderedList.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUnorderedList.java
index 764d05b..459ccf5 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUnorderedList.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUnorderedList.java
@@ -16,6 +16,10 @@
import java.util.ArrayList;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.AUnorderedListType;
import edu.uci.ics.asterix.om.types.IAType;
@@ -107,4 +111,17 @@
sb.append(" ]");
return sb.toString();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ JSONArray list = new JSONArray();
+ for (IAObject v : values) {
+ list.put(v.toJSON());
+ }
+ json.put("AUnorderedList", list);
+
+ return json;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/IAObject.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/IAObject.java
index 463154b..bfc0aba 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/IAObject.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/IAObject.java
@@ -17,11 +17,12 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.visitors.IOMVisitor;
+import edu.uci.ics.hyracks.api.dataflow.value.JSONSerializable;
/**
* Represents an object in Asterix.
*/
-public interface IAObject {
+public interface IAObject extends JSONSerializable {
public IAType getType();
public void accept(IOMVisitor visitor) throws AsterixException;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AOrderedListType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AOrderedListType.java
index 59423e2..5620e4e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AOrderedListType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AOrderedListType.java
@@ -1,5 +1,8 @@
package edu.uci.ics.asterix.om.types;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.om.base.IAObject;
public class AOrderedListType extends AbstractCollectionType {
@@ -52,4 +55,11 @@
public int hash() {
return hashCode();
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException{
+ JSONObject type = new JSONObject();
+ type.put("type", itemType);
+ return type;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index ac1facc..20e3f56 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -22,6 +22,10 @@
import java.util.Arrays;
import java.util.List;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.annotations.IRecordTypeAnnotation;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.IAObject;
@@ -244,4 +248,23 @@
}
return h;
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ if (isOpen) {
+ type.put("open", true);
+ } else {
+ type.put("open", false);
+ }
+
+ JSONArray fields = new JSONArray();
+ for (int i = 0; i < fieldNames.length; i++) {
+ JSONObject field = new JSONObject();
+ field.put(fieldNames[i], fieldTypes[i].toJSON());
+ fields.put(field);
+ }
+
+ type.put("fields", fields);
+ return type;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnionType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnionType.java
index 1e70f15..dc74b6e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnionType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnionType.java
@@ -4,6 +4,10 @@
import java.util.Iterator;
import java.util.List;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.IAObject;
import edu.uci.ics.asterix.om.visitors.IOMVisitor;
@@ -27,7 +31,7 @@
}
public boolean isNullableType() {
- return unionList.size() == 2 && unionList.get(0).equals(BuiltinType.ANULL);
+ return unionList.size() == 2 && unionList.get(0).equals(BuiltinType.ANULL);
}
@Override
@@ -100,4 +104,23 @@
return h;
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "UNION");
+
+ JSONArray fields = new JSONArray();
+
+ Iterator<IAType> iter = unionList.iterator();
+ if (iter.hasNext()) {
+ IAType t0 = iter.next();
+ fields.put(t0.toJSON());
+ while (iter.hasNext()) {
+ fields.put(iter.next().toJSON());
+ }
+ }
+
+ type.put("fields", fields);
+ return type;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnorderedListType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnorderedListType.java
index c15dc49..a62bd5b 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnorderedListType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/AUnorderedListType.java
@@ -1,5 +1,8 @@
package edu.uci.ics.asterix.om.types;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.om.base.IAObject;
public class AUnorderedListType extends AbstractCollectionType {
@@ -53,4 +56,10 @@
return hashCode();
}
+ @Override
+ public JSONObject toJSON() throws JSONException{
+ JSONObject type = new JSONObject();
+ type.put("type", itemType);
+ return type;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
index 4d9cd7f..1e498bc 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
@@ -1,5 +1,8 @@
package edu.uci.ics.asterix.om.types;
+import org.json.JSONException;
+import org.json.JSONObject;
+
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.IAObject;
import edu.uci.ics.asterix.om.visitors.IOMVisitor;
@@ -41,6 +44,13 @@
public String getConstructor() {
return null;
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AsterixType");
+ return type;
+ }
};
public final static BuiltinType AINT8 = new LowerCaseConstructorType() {
@@ -61,6 +71,13 @@
public String getTypeName() {
return "int8";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AInt8");
+ return type;
+ }
};
public final static BuiltinType AINT16 = new LowerCaseConstructorType() {
@@ -81,6 +98,13 @@
public String getTypeName() {
return "int16";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AInt16");
+ return type;
+ }
};
public final static BuiltinType AINT32 = new LowerCaseConstructorType() {
@@ -101,6 +125,13 @@
public String getTypeName() {
return "int32";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AInt32");
+ return type;
+ }
};
public static final BuiltinType AINT64 = new LowerCaseConstructorType() {
@@ -122,6 +153,12 @@
return "int64";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AInt64");
+ return type;
+ }
};
public final static BuiltinType ABINARY = new LowerCaseConstructorType() {
@@ -143,6 +180,12 @@
return "binary";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ABinary");
+ return type;
+ }
};
public final static BuiltinType AFLOAT = new LowerCaseConstructorType() {
@@ -164,6 +207,12 @@
return "float";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AFloat");
+ return type;
+ }
};
public final static BuiltinType ADOUBLE = new LowerCaseConstructorType() {
@@ -184,6 +233,13 @@
public String getTypeName() {
return "double";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ADouble");
+ return type;
+ }
};
public final static BuiltinType ASTRING = new LowerCaseConstructorType() {
@@ -204,6 +260,13 @@
public String getTypeName() {
return "string";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "AString");
+ return type;
+ }
};
public final static BuiltinType ANULL = new LowerCaseConstructorType() {
@@ -225,6 +288,12 @@
return "null";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "Null");
+ return type;
+ }
};
public final static BuiltinType ABOOLEAN = new LowerCaseConstructorType() {
@@ -246,6 +315,12 @@
return "boolean";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ABoolean");
+ return type;
+ }
};
public final static BuiltinType ATIME = new LowerCaseConstructorType() {
@@ -266,6 +341,13 @@
public String getTypeName() {
return "time";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ATime");
+ return type;
+ }
};
public final static BuiltinType ADATE = new LowerCaseConstructorType() {
@@ -286,6 +368,13 @@
public String getTypeName() {
return "date";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ADate");
+ return type;
+ }
};
public final static BuiltinType ADATETIME = new LowerCaseConstructorType() {
@@ -306,6 +395,13 @@
public String getTypeName() {
return "datetime";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ADateTime");
+ return type;
+ }
};
public final static BuiltinType ADURATION = new LowerCaseConstructorType() {
@@ -327,6 +423,12 @@
return "duration";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ADuration");
+ return type;
+ }
};
public final static BuiltinType AINTERVAL = new LowerCaseConstructorType() {
@@ -368,6 +470,13 @@
public String getTypeName() {
return "point";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "APoint");
+ return type;
+ }
};
public final static BuiltinType APOINT3D = new LowerCaseConstructorType() {
@@ -388,6 +497,13 @@
public String getTypeName() {
return "point3d";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "APoint3D");
+ return type;
+ }
};
public final static BuiltinType ALINE = new LowerCaseConstructorType() {
@@ -409,6 +525,12 @@
return "line";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ALINE");
+ return type;
+ }
};
public final static BuiltinType APOLYGON = new LowerCaseConstructorType() {
@@ -430,6 +552,12 @@
return "polygon";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "APOLYGON");
+ return type;
+ }
};
public final static BuiltinType ACIRCLE = new LowerCaseConstructorType() {
@@ -451,6 +579,12 @@
return "circle";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ACIRCLE");
+ return type;
+ }
};
public final static BuiltinType ARECTANGLE = new LowerCaseConstructorType() {
@@ -472,6 +606,12 @@
return "rectangle";
}
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ARECTANGLE");
+ return type;
+ }
};
public static final IAType ABITARRAY = new LowerCaseConstructorType() {
@@ -492,6 +632,13 @@
public String getTypeName() {
return "abitarray";
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ABitArray");
+ return type;
+ }
};
public static final IAType ANY = new BuiltinType() {
@@ -517,6 +664,13 @@
public String getConstructor() {
return null;
}
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "ANY");
+ return type;
+ }
};
public abstract String getConstructor();