You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org> on 2016/02/21 13:23:20 UTC

Change in asterixdb[master]: Add Test Servers to Test Framework

abdullah alamoudi has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/651

Change subject: Add Test Servers to Test Framework
......................................................................

Add Test Servers to Test Framework

This change introduces the ability to start and stop
different servers for any purpose, especially for testing
External data sources and different types of feeds.

Change-Id: I3d0434925972770cdae168656e1672cf0f225980
---
M asterix-common/pom.xml
M asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
A asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
7 files changed, 389 insertions(+), 0 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/51/651/1

diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 02f651f..7eb2329 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -233,6 +233,11 @@
 			<artifactId>jackson-mapper-asl</artifactId>
 			<version>1.9.12</version>
 		</dependency>
+        <dependency>
+            <groupId>com.rometools</groupId>
+            <artifactId>rome</artifactId>
+            <version>1.5.1</version>
+        </dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index 29e08b5..673f212 100644
--- a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -31,6 +31,8 @@
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -39,6 +41,8 @@
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.utils.ServletUtil.Servlets;
+import org.apache.asterix.test.server.ITestServer;
+import org.apache.asterix.test.server.TestServerProvider;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.asterix.testframework.context.TestFileContext;
@@ -65,6 +69,8 @@
     // https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
     private static final long MAX_URL_LENGTH = 2000l;
     private static Method managixExecuteMethod = null;
+
+    private HashMap<Integer, ITestServer> testServers = new HashMap<>();
 
     private String host;
     private int port;
@@ -633,6 +639,51 @@
                                 throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
                             }
                             break;
+                        case "server": //(start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))
+                            try {
+                                String[] command = statement.trim().split(" ");
+                                if (command.length < 2) {
+                                    throw new Exception(
+                                            "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
+                                }
+                                String action = command[0];
+                                if (action.equals("start")) {
+                                    if (command.length < 3) {
+                                        throw new Exception(
+                                                "invalid server start command. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...");
+                                    }
+                                    String name = command[1];
+                                    Integer port = new Integer(command[2]);
+                                    if (testServers.containsKey(port)) {
+                                        throw new Exception("server with port " + port + " is already running");
+                                    }
+                                    ITestServer server = TestServerProvider.createTestServer(name, port);
+                                    server.configure(Arrays.copyOfRange(command, 3, command.length));
+                                    server.start();
+                                    testServers.put(port, server);
+                                } else if (action.equals("stop")) {
+                                    String target = command[1];
+                                    if (target.equals("all")) {
+                                        for (ITestServer server : testServers.values()) {
+                                            server.stop();
+                                        }
+                                        testServers.clear();
+                                    } else {
+                                        Integer port = new Integer(command[1]);
+                                        ITestServer server = testServers.get(port);
+                                        if (server == null) {
+                                            throw new Exception("no server is listening to port " + port);
+                                        }
+                                        server.stop();
+                                        testServers.remove(port);
+                                    }
+                                } else {
+                                    throw new Exception("unknown server action");
+                                }
+                            } catch (Exception e) {
+                                throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+                            }
+                            break;
                         default:
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java b/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
new file mode 100644
index 0000000..9310ea6
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+
+public class FileTestServer implements ITestServer {
+    private String[] paths;
+    private int port;
+    private ServerSocket serverSocket;
+
+    public FileTestServer(int port) {
+        this.port = port;
+    }
+
+    @Override
+    public void configure(String[] args) {
+        this.paths = args;
+    }
+
+    @Override
+    public void start() throws IOException {
+        serverSocket = new ServerSocket(port);
+        while (!serverSocket.isClosed()) {
+            Socket socket = null;
+            try {
+                socket = serverSocket.accept();
+                OutputStream os = socket.getOutputStream();
+                for (String path : paths) {
+                    FileInputStream fin = new FileInputStream(new File(path));
+                    byte[] chunk = new byte[1024];
+                    int read = fin.read(chunk);
+                    try {
+                        while (read > 0) {
+                            os.write(chunk, 0, read);
+                            read = fin.read(chunk);
+                        }
+                    } finally {
+                        fin.close();
+                    }
+                }
+            } catch (IOException e) {
+            } finally {
+                socket.close();
+            }
+        }
+    }
+
+    @Override
+    public void stop() throws IOException {
+        serverSocket.close();
+    }
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java b/asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
new file mode 100644
index 0000000..18a4969
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+public interface ITestServer {
+
+    public void configure(String[] args);
+
+    public void start() throws Exception;
+
+    public void stop() throws Exception;
+
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
new file mode 100644
index 0000000..17646b5
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import com.rometools.rome.feed.synd.SyndContent;
+import com.rometools.rome.feed.synd.SyndContentImpl;
+import com.rometools.rome.feed.synd.SyndEntry;
+import com.rometools.rome.feed.synd.SyndEntryImpl;
+import com.rometools.rome.feed.synd.SyndFeed;
+import com.rometools.rome.feed.synd.SyndFeedImpl;
+import com.rometools.rome.io.FeedException;
+import com.rometools.rome.io.SyndFeedOutput;
+
+public class RSSFeedServlet extends HttpServlet {
+
+    private static final long serialVersionUID = 1L;
+    private static final String DEFAULT_FEED_TYPE = "default.feed.type";
+    private static final String FEED_TYPE = "type";
+    private static final String MIME_TYPE = "application/xml; charset=UTF-8";
+    private static final String COULD_NOT_GENERATE_FEED_ERROR = "Could not generate feed";
+
+    private static final DateFormat DATE_PARSER = new SimpleDateFormat("yyyy-MM-dd");
+    private String _defaultFeedType;
+
+    @Override
+    public void init() {
+        _defaultFeedType = getServletConfig().getInitParameter(DEFAULT_FEED_TYPE);
+        _defaultFeedType = (_defaultFeedType != null) ? _defaultFeedType : "atom_0.3";
+    }
+
+    @Override
+    public void doGet(HttpServletRequest req, HttpServletResponse res) throws IOException {
+        try {
+            SyndFeed feed = getFeed(req);
+            String feedType = req.getParameter(FEED_TYPE);
+            feedType = (feedType != null) ? feedType : _defaultFeedType;
+            feed.setFeedType(feedType);
+            res.setContentType(MIME_TYPE);
+            SyndFeedOutput output = new SyndFeedOutput();
+            output.output(feed, res.getWriter());
+        } catch (FeedException ex) {
+            String msg = COULD_NOT_GENERATE_FEED_ERROR;
+            log(msg, ex);
+            res.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, msg);
+        }
+    }
+
+    protected SyndFeed getFeed(HttpServletRequest req) throws IOException, FeedException {
+        SyndFeed feed = new SyndFeedImpl();
+        feed.setTitle("Sample Feed (created with ROME)");
+        feed.setLink("http://rome.dev.java.net");
+        feed.setDescription("This feed has been created using ROME (Java syndication utilities");
+
+        List<SyndEntry> entries = new ArrayList<SyndEntry>();
+        SyndEntry entry;
+        SyndContent description;
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("AsterixDB 0.8.7");
+        entry.setLink("http://http://asterixdb.apache.org/docs/0.8.7-incubating/index.html");
+        try {
+            entry.setPublishedDate(DATE_PARSER.parse("2012-06-08"));
+        } catch (ParseException ex) {
+            // IT CANNOT HAPPEN WITH THIS SAMPLE
+        }
+        description = new SyndContentImpl();
+        description.setType("text/plain");
+        description.setValue("AsterixDB 0.8.7 Release");
+        entry.setDescription(description);
+        entries.add(entry);
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("Couchbase 4.1");
+        entry.setLink("http://blog.couchbase.com/2015/december/introducing-couchbase-server-4.1");
+        try {
+            entry.setPublishedDate(DATE_PARSER.parse("2015-12-09"));
+        } catch (ParseException ex) {
+            // IT CANNOT HAPPEN WITH THIS SAMPLE
+        }
+        description = new SyndContentImpl();
+        description.setType("text/plain");
+        description.setValue("Couchbase Server 4.1 Release. Bug fixes, minor API changes and some new features");
+        entry.setDescription(description);
+        entries.add(entry);
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("ROME v0.3");
+        entry.setLink("http://wiki.java.net/bin/view/Javawsxml/rome03");
+        try {
+            entry.setPublishedDate(DATE_PARSER.parse("2004-07-27"));
+        } catch (ParseException ex) {
+            // IT CANNOT HAPPEN WITH THIS SAMPLE
+        }
+        description = new SyndContentImpl();
+        description.setType("text/html");
+        description.setValue("<p>Bug fixes, API changes, some new features and some Unit testing</p>"
+                + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#ChangeLog-Changesmadefromv0.3tov0.4\">Changes Log for 0.3</a></p>");
+        entry.setDescription(description);
+        entries.add(entry);
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("ROME v0.4");
+        entry.setLink("http://wiki.java.net/bin/view/Javawsxml/rome04");
+        try {
+            entry.setPublishedDate(DATE_PARSER.parse("2004-09-24"));
+        } catch (ParseException ex) {
+            // IT CANNOT HAPPEN WITH THIS SAMPLE
+        }
+        description = new SyndContentImpl();
+        description.setType("text/html");
+        description.setValue("<p>Bug fixes, API changes, some new features, Unit testing completed</p>"
+                + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#ChangeLog-Changesmadefromv0.4tov0.5\">Changes Log for 0.4</a></p>");
+        entry.setDescription(description);
+        entries.add(entry);
+        feed.setEntries(entries);
+        return feed;
+    }
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
new file mode 100644
index 0000000..5855e1e
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+public class RSSTestServer implements ITestServer {
+
+    private Server rssServer;
+
+    public RSSTestServer(int port) {
+        rssServer = new Server(port);
+        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+        context.setContextPath("/");
+        rssServer.setHandler(context);
+        context.addServlet(new ServletHolder(new RSSFeedServlet()), "/");
+    }
+
+    @Override
+    public void configure(String[] args) {
+    }
+
+    @Override
+    public void start() throws Exception {
+        rssServer.start();
+    }
+
+    @Override
+    public void stop() throws Exception {
+        rssServer.stop();
+        rssServer.join();
+    }
+
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java b/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
new file mode 100644
index 0000000..29e55f7
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+public class TestServerProvider {
+
+    public static ITestServer createTestServer(String string, Integer port) throws Exception {
+        switch (string) {
+            case "file":
+                return new FileTestServer(port);
+            case "rss":
+                return new RSSTestServer(port);
+            default:
+                throw new Exception("Unknown test server");
+        }
+    }
+}
\ No newline at end of file

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 2:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/651/2/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 648:                                             "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
> This is still very long, can we break it down to fit in a line?
Sure but the comment I received was about the comment, not the string :)


Line 654:                                                 "invalid server start command. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...");
> This is still very long, can we break it down to fit in a line?
Sure but the comment I received was about the comment, not the string :)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/857/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/651

to look at the new patch set (#3).

Change subject: Add Test Servers to Test Framework
......................................................................

Add Test Servers to Test Framework

This change introduces the ability to start and stop
different servers for any purpose, especially for testing
External data sources and different types of feeds.

Change-Id: I3d0434925972770cdae168656e1672cf0f225980
---
M asterix-common/pom.xml
M asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
A asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
7 files changed, 395 insertions(+), 12 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/51/651/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/857/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 3: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
File asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java:

Line 91:             // IT CANNOT HAPPEN WITH THIS SAMPLE
> Let's throw unchecked exceptions here - this is test code.
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/865/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 1:

(6 comments)

https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 73:     private HashMap<Integer, ITestServer> testServers = new HashMap<>();
static final. Rename to testRunningServers


Line 662:                                     server.start();
this start call in case of FileTestServer is going to block due to server.accept() right? If that's the case, then the socket should be started on a different thread.


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
File asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java:

Line 30:     private int port;
final


Line 52:                     byte[] chunk = new byte[1024];
how about?
while (!serverSocket.isClosed()) {
            try (Socket socket = serverSocket.accept(); OutputStream os = socket.getOutputStream()) {
                byte[] chunk = new byte[1024];
                for (String path : paths) {
                    try (FileInputStream fin = new FileInputStream(new File(path))) {
                        int read = fin.read(chunk);
                        while (read > 0) {
                            os.write(chunk, 0, read);
                            read = fin.read(chunk);
                        }
                    }
                }
            }
        }


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
File asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java:

Line 50:     private String _defaultFeedType;
since when do we name member variables like this? :)


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
File asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java:

Line 23:     public static ITestServer createTestServer(String string, Integer port) throws Exception {
string -> name


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has submitted this change and it was merged.

Change subject: Add Test Servers to Test Framework
......................................................................


Add Test Servers to Test Framework

This change introduces the ability to start and stop
different servers for any purpose, especially for testing
External data sources and different types of feeds.

Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Reviewed-on: https://asterix-gerrit.ics.uci.edu/651
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <hu...@gmail.com>
---
M asterix-common/pom.xml
M asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
A asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
7 files changed, 395 insertions(+), 12 deletions(-)

Approvals:
  Murtadha Hubail: Looks good to me, approved
  Jenkins: Verified



diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 02f651f..7eb2329 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -233,6 +233,11 @@
 			<artifactId>jackson-mapper-asl</artifactId>
 			<version>1.9.12</version>
 		</dependency>
+        <dependency>
+            <groupId>com.rometools</groupId>
+            <artifactId>rome</artifactId>
+            <version>1.5.1</version>
+        </dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index 29e08b5..ae8c9cb 100644
--- a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -31,6 +31,8 @@
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -39,6 +41,8 @@
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.utils.ServletUtil.Servlets;
+import org.apache.asterix.test.server.ITestServer;
+import org.apache.asterix.test.server.TestServerProvider;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.asterix.testframework.context.TestFileContext;
@@ -65,6 +69,7 @@
     // https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
     private static final long MAX_URL_LENGTH = 2000l;
     private static Method managixExecuteMethod = null;
+    private static final HashMap<Integer, ITestServer> runningTestServers = new HashMap<>();
 
     private String host;
     private int port;
@@ -237,7 +242,7 @@
             throws Exception {
         HttpMethodBase method = null;
         if (str.length() + url.length() < MAX_URL_LENGTH) {
-            //Use GET for small-ish queries
+            // Use GET for small-ish queries
             method = new GetMethod(url);
             NameValuePair[] parameters = new NameValuePair[params.size() + 1];
             parameters[0] = new NameValuePair("query", str);
@@ -247,13 +252,13 @@
             }
             method.setQueryString(parameters);
         } else {
-            //Use POST for bigger ones to avoid 413 FULL_HEAD
+            // Use POST for bigger ones to avoid 413 FULL_HEAD
             // QQQ POST API doesn't allow encoding additional parameters
             method = new PostMethod(url);
             ((PostMethod) method).setRequestEntity(new StringRequestEntity(str));
         }
 
-        //Set accepted output response type
+        // Set accepted output response type
         method.setRequestHeader("Accept", fmt.mimeType());
         // Provide custom retry handler is necessary
         method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, new DefaultHttpMethodRetryHandler(3, false));
@@ -264,7 +269,7 @@
     public InputStream executeClusterStateQuery(OutputFormat fmt, String url) throws Exception {
         HttpMethodBase method = new GetMethod(url);
 
-        //Set accepted output response type
+        // Set accepted output response type
         method.setRequestHeader("Accept", fmt.mimeType());
         // Provide custom retry handler is necessary
         method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, new DefaultHttpMethodRetryHandler(3, false));
@@ -463,7 +468,7 @@
                             }
                             break;
                         case "update":
-                            //isDmlRecoveryTest: set IP address
+                            // isDmlRecoveryTest: set IP address
                             if (isDmlRecoveryTest && statement.contains("nc1://")) {
                                 statement = statement.replaceAll("nc1://",
                                         "127.0.0.1://../../../../../../asterix-app/");
@@ -531,7 +536,7 @@
                         case "mgx":
                             executeManagixCommand(statement);
                             break;
-                        case "txnqbc": //qbc represents query before crash
+                        case "txnqbc": // qbc represents query before crash
                             resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
                                     "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(), cUnit.getParameter());
                             qbcFile = new File(actualPath + File.separator
@@ -540,7 +545,7 @@
                             qbcFile.getParentFile().mkdirs();
                             writeOutputToFile(qbcFile, resultStream);
                             break;
-                        case "txnqar": //qar represents query after recovery
+                        case "txnqar": // qar represents query after recovery
                             resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
                                     "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(), cUnit.getParameter());
                             qarFile = new File(actualPath + File.separator
@@ -550,11 +555,11 @@
                             writeOutputToFile(qarFile, resultStream);
                             runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), qbcFile, qarFile);
                             break;
-                        case "txneu": //eu represents erroneous update
+                        case "txneu": // eu represents erroneous update
                             try {
                                 executeUpdate(statement, "http://" + host + ":" + port + Servlets.AQL_UPDATE.getPath());
                             } catch (Exception e) {
-                                //An exception is expected.
+                                // An exception is expected.
                                 failed = true;
                                 e.printStackTrace();
                             }
@@ -591,7 +596,7 @@
                             }
                             System.err.println("...but that was expected.");
                             break;
-                        case "vscript": //a script that will be executed on a vagrant virtual node
+                        case "vscript": // a script that will be executed on a vagrant virtual node
                             try {
                                 String[] command = statement.trim().split(" ");
                                 if (command.length != 2) {
@@ -607,7 +612,7 @@
                                 throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
                             }
                             break;
-                        case "vmgx": //a managix command that will be executed on vagrant cc node
+                        case "vmgx": // a managix command that will be executed on vagrant cc node
                             try {
                                 String output = executeVagrantManagix(pb, statement);
                                 if (output.contains("ERROR")) {
@@ -617,7 +622,7 @@
                                 throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
                             }
                             break;
-                        case "cstate": //cluster state query
+                        case "cstate": // cluster state query
                             try {
                                 fmt = OutputFormat.forCompilationUnit(cUnit);
                                 resultStream = executeClusterStateQuery(fmt,
@@ -633,6 +638,54 @@
                                 throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
                             }
                             break;
+                        case "server": // (start <test server name> <port>
+                                       // [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))
+                            try {
+                                String[] lines = statement.trim().split("\n");
+                                String[] command = lines[lines.length - 1].trim().split(" ");
+                                if (command.length < 2) {
+                                    throw new Exception("invalid server command format. expected format ="
+                                            + " (start <test server name> <port> [<arg1>][<arg2>][<arg3>]"
+                                            + "...|stop (<port>|all))");
+                                }
+                                String action = command[0];
+                                if (action.equals("start")) {
+                                    if (command.length < 3) {
+                                        throw new Exception("invalid server start command. expected format ="
+                                                + " (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...");
+                                    }
+                                    String name = command[1];
+                                    Integer port = new Integer(command[2]);
+                                    if (runningTestServers.containsKey(port)) {
+                                        throw new Exception("server with port " + port + " is already running");
+                                    }
+                                    ITestServer server = TestServerProvider.createTestServer(name, port);
+                                    server.configure(Arrays.copyOfRange(command, 3, command.length));
+                                    server.start();
+                                    runningTestServers.put(port, server);
+                                } else if (action.equals("stop")) {
+                                    String target = command[1];
+                                    if (target.equals("all")) {
+                                        for (ITestServer server : runningTestServers.values()) {
+                                            server.stop();
+                                        }
+                                        runningTestServers.clear();
+                                    } else {
+                                        Integer port = new Integer(command[1]);
+                                        ITestServer server = runningTestServers.get(port);
+                                        if (server == null) {
+                                            throw new Exception("no server is listening to port " + port);
+                                        }
+                                        server.stop();
+                                        runningTestServers.remove(port);
+                                    }
+                                } else {
+                                    throw new Exception("unknown server action");
+                                }
+                            } catch (Exception e) {
+                                throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+                            }
+                            break;
                         default:
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java b/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
new file mode 100644
index 0000000..f40cce4
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+
+public class FileTestServer implements ITestServer {
+    private String[] paths;
+    private final int port;
+    private ServerSocket serverSocket;
+    private Thread listenerThread;
+
+    public FileTestServer(int port) {
+        this.port = port;
+    }
+
+    @Override
+    public void configure(String[] args) {
+        this.paths = args;
+    }
+
+    @Override
+    public void start() throws IOException {
+        serverSocket = new ServerSocket(port);
+        listenerThread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                while (!serverSocket.isClosed()) {
+                    try (Socket socket = serverSocket.accept(); OutputStream os = socket.getOutputStream()) {
+                        byte[] chunk = new byte[1024];
+                        for (String path : paths) {
+                            try (FileInputStream fin = new FileInputStream(new File(path))) {
+                                int read = fin.read(chunk);
+                                while (read > 0) {
+                                    os.write(chunk, 0, read);
+                                    read = fin.read(chunk);
+                                }
+                            }
+                        }
+                    } catch (IOException e) {
+                        // Do nothing. This means the socket was closed for some reason.
+                        // There is nothing to do here except try to close the socket and see if the
+                        // server is still listening!
+                        // This also could be due to the close() call
+                    }
+                }
+            }
+        });
+        listenerThread.start();
+    }
+
+    @Override
+    public void stop() throws IOException, InterruptedException {
+        serverSocket.close();
+        if (listenerThread.isAlive()) {
+            listenerThread.join();
+        }
+    }
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java b/asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
new file mode 100644
index 0000000..18a4969
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+public interface ITestServer {
+
+    public void configure(String[] args);
+
+    public void start() throws Exception;
+
+    public void stop() throws Exception;
+
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
new file mode 100644
index 0000000..f025017
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import com.rometools.rome.feed.synd.SyndContent;
+import com.rometools.rome.feed.synd.SyndContentImpl;
+import com.rometools.rome.feed.synd.SyndEntry;
+import com.rometools.rome.feed.synd.SyndEntryImpl;
+import com.rometools.rome.feed.synd.SyndFeed;
+import com.rometools.rome.feed.synd.SyndFeedImpl;
+import com.rometools.rome.io.FeedException;
+import com.rometools.rome.io.SyndFeedOutput;
+
+public class RSSFeedServlet extends HttpServlet {
+
+    private static final long serialVersionUID = 1L;
+    private static final String DEFAULT_FEED_TYPE = "default.feed.type";
+    private static final String FEED_TYPE = "type";
+    private static final String MIME_TYPE = "application/xml; charset=UTF-8";
+    private static final String COULD_NOT_GENERATE_FEED_ERROR = "Could not generate feed";
+
+    private static final DateFormat DATE_PARSER = new SimpleDateFormat("yyyy-MM-dd");
+    private String defaultFeedType;
+
+    @Override
+    public void init() {
+        defaultFeedType = getServletConfig().getInitParameter(DEFAULT_FEED_TYPE);
+        defaultFeedType = (defaultFeedType != null) ? defaultFeedType : "atom_0.3";
+    }
+
+    @Override
+    public void doGet(HttpServletRequest req, HttpServletResponse res) throws IOException {
+        try {
+            SyndFeed feed = getFeed(req);
+            String feedType = req.getParameter(FEED_TYPE);
+            feedType = (feedType != null) ? feedType : defaultFeedType;
+            feed.setFeedType(feedType);
+            res.setContentType(MIME_TYPE);
+            SyndFeedOutput output = new SyndFeedOutput();
+            output.output(feed, res.getWriter());
+        } catch (FeedException | ParseException ex) {
+            String msg = COULD_NOT_GENERATE_FEED_ERROR;
+            log(msg, ex);
+            res.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, msg);
+        }
+    }
+
+    protected SyndFeed getFeed(HttpServletRequest req) throws IOException, FeedException, ParseException {
+        SyndFeed feed = new SyndFeedImpl();
+        feed.setTitle("Sample Feed (created with ROME)");
+        feed.setLink("http://rome.dev.java.net");
+        feed.setDescription("This feed has been created using ROME (Java syndication utilities");
+
+        List<SyndEntry> entries = new ArrayList<SyndEntry>();
+        SyndEntry entry;
+        SyndContent description;
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("AsterixDB 0.8.7");
+        entry.setLink("http://http://asterixdb.apache.org/docs/0.8.7-incubating/index.html");
+        entry.setPublishedDate(DATE_PARSER.parse("2012-06-08"));
+        description = new SyndContentImpl();
+        description.setType("text/plain");
+        description.setValue("AsterixDB 0.8.7 Release");
+        entry.setDescription(description);
+        entries.add(entry);
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("Couchbase 4.1");
+        entry.setLink("http://blog.couchbase.com/2015/december/introducing-couchbase-server-4.1");
+        entry.setPublishedDate(DATE_PARSER.parse("2015-12-09"));
+        description = new SyndContentImpl();
+        description.setType("text/plain");
+        description.setValue("Couchbase Server 4.1 Release. Bug fixes, minor API changes and some new features");
+        entry.setDescription(description);
+        entries.add(entry);
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("ROME v0.3");
+        entry.setLink("http://wiki.java.net/bin/view/Javawsxml/rome03");
+        entry.setPublishedDate(DATE_PARSER.parse("2004-07-27"));
+        description = new SyndContentImpl();
+        description.setType("text/html");
+        description.setValue("<p>Bug fixes, API changes, some new features and some Unit testing</p>"
+                + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#"
+                + "ChangeLog-Changesmadefromv0.3tov0.4\">Changes Log for 0.3</a></p>");
+        entry.setDescription(description);
+        entries.add(entry);
+
+        entry = new SyndEntryImpl();
+        entry.setTitle("ROME v0.4");
+        entry.setLink("http://wiki.java.net/bin/view/Javawsxml/rome04");
+        entry.setPublishedDate(DATE_PARSER.parse("2004-09-24"));
+        description = new SyndContentImpl();
+        description.setType("text/html");
+        description.setValue("<p>Bug fixes, API changes, some new features, Unit testing completed</p>"
+                + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#"
+                + "ChangeLog-Changesmadefromv0.4tov0.5\">Changes Log for 0.4</a></p>");
+        entry.setDescription(description);
+        entries.add(entry);
+        feed.setEntries(entries);
+        return feed;
+    }
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
new file mode 100644
index 0000000..5855e1e
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+public class RSSTestServer implements ITestServer {
+
+    private Server rssServer;
+
+    public RSSTestServer(int port) {
+        rssServer = new Server(port);
+        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+        context.setContextPath("/");
+        rssServer.setHandler(context);
+        context.addServlet(new ServletHolder(new RSSFeedServlet()), "/");
+    }
+
+    @Override
+    public void configure(String[] args) {
+    }
+
+    @Override
+    public void start() throws Exception {
+        rssServer.start();
+    }
+
+    @Override
+    public void stop() throws Exception {
+        rssServer.stop();
+        rssServer.join();
+    }
+
+}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java b/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
new file mode 100644
index 0000000..0be6800
--- /dev/null
+++ b/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+public class TestServerProvider {
+
+    public static ITestServer createTestServer(String name, Integer port) throws Exception {
+        switch (name) {
+            case "file":
+                return new FileTestServer(port);
+            case "rss":
+                return new RSSTestServer(port);
+            default:
+                throw new Exception("Unknown test server");
+        }
+    }
+}
\ No newline at end of file

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 1:

(11 comments)

Great step towards better test coverage!

https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/pom.xml
File asterix-common/pom.xml:

Line 240:         </dependency>
Please add an issue to reflect this in the LICENSE/NOTICE


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 644:                                 String[] command = statement.trim().split(" ");
Would be nice to skip lines starting with a '#' here, so that we can add comments/license headers.


Line 647:                                             "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
line break


Line 653:                                                 "invalid server start command. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...");
line break


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
File asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java:

Line 63:             } catch (IOException e) {
Why do we ignore exceptions here? Please add a comment.


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
File asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java:

Line 91:             // IT CANNOT HAPPEN WITH THIS SAMPLE
Let's throw unchecked exceptions here - this is test code.


Line 105:             // IT CANNOT HAPPEN WITH THIS SAMPLE
Let's throw unchecked exceptions here - this is test code.


Line 119:             // IT CANNOT HAPPEN WITH THIS SAMPLE
Let's throw unchecked exceptions here - this is test code.


Line 124:                 + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#ChangeLog-Changesmadefromv0.3tov0.4\">Changes Log for 0.3</a></p>");
line break


Line 134:             // IT CANNOT HAPPEN WITH THIS SAMPLE
Let's throw unchecked exceptions here - this is test code.


Line 139:                 + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#ChangeLog-Changesmadefromv0.4tov0.5\">Changes Log for 0.4</a></p>");
line break


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/651/2/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 648:                                             "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
> Sure but the comment I received was about the comment, not the string :)
But there is no comment here (and there never was ...)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 3: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/865/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 2:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/651/2/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 648:                                             "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
This is still very long, can we break it down to fit in a line?


Line 654:                                                 "invalid server start command. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...");
This is still very long, can we break it down to fit in a line?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/651

to look at the new patch set (#2).

Change subject: Add Test Servers to Test Framework
......................................................................

Add Test Servers to Test Framework

This change introduces the ability to start and stop
different servers for any purpose, especially for testing
External data sources and different types of feeds.

Change-Id: I3d0434925972770cdae168656e1672cf0f225980
---
M asterix-common/pom.xml
M asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
A asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/ITestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
A asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
A asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
7 files changed, 394 insertions(+), 12 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/51/651/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 2: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/861/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 1:

(16 comments)

https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/pom.xml
File asterix-common/pom.xml:

Line 240:         </dependency>
> Please add an issue to reflect this in the LICENSE/NOTICE
Done


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 73:     private HashMap<Integer, ITestServer> testServers = new HashMap<>();
> static final. Rename to testRunningServers
Done


Line 644:                                 String[] command = statement.trim().split(" ");
> Would be nice to skip lines starting with a '#' here, so that we can add co
Done


Line 647:                                             "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
> line break
Done


Line 653:                                                 "invalid server start command. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...");
> line break
Done


Line 662:                                     server.start();
> this start call in case of FileTestServer is going to block due to server.a
Good catch.
Done


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java
File asterix-common/src/test/java/org/apache/asterix/test/server/FileTestServer.java:

Line 30:     private int port;
> final
Done


Line 52:                     byte[] chunk = new byte[1024];
> how about?
I can't handle this much coolness!
Done


Line 63:             } catch (IOException e) {
> Why do we ignore exceptions here? Please add a comment.
Done


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java
File asterix-common/src/test/java/org/apache/asterix/test/server/RSSFeedServlet.java:

Line 50:     private String _defaultFeedType;
> since when do we name member variables like this? :)
You know that I know that you know that I copied this from a sample example :D


Line 105:             // IT CANNOT HAPPEN WITH THIS SAMPLE
> Let's throw unchecked exceptions here - this is test code.
Done


Line 119:             // IT CANNOT HAPPEN WITH THIS SAMPLE
> Let's throw unchecked exceptions here - this is test code.
Done


Line 124:                 + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#ChangeLog-Changesmadefromv0.3tov0.4\">Changes Log for 0.3</a></p>");
> line break
Done


Line 134:             // IT CANNOT HAPPEN WITH THIS SAMPLE
> Let's throw unchecked exceptions here - this is test code.
Done


Line 139:                 + "<p>For details check the <a href=\"https://rometools.jira.com/wiki/display/ROME/Change+Log#ChangeLog-Changesmadefromv0.4tov0.5\">Changes Log for 0.4</a></p>");
> line break
Done


https://asterix-gerrit.ics.uci.edu/#/c/651/1/asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java
File asterix-common/src/test/java/org/apache/asterix/test/server/TestServerProvider.java:

Line 23:     public static ITestServer createTestServer(String string, Integer port) throws Exception {
> string -> name
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/861/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Add Test Servers to Test Framework

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: Add Test Servers to Test Framework
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/651/2/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
File asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java:

Line 648:                                             "invalid server command format. expected format = (start <test server name> <port> [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))");
> But there is no comment here (and there never was ...)
Mmm, for some reason, I thought there was :D


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/651
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I3d0434925972770cdae168656e1672cf0f225980
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes