You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by bl...@apache.org on 2012/09/07 21:48:36 UTC

[3/3] SQOOP-596: Implement connection resource end-to-end

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
index c112843..3cfc7f3 100644
--- a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
+++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
@@ -210,8 +210,20 @@ public final class DerbySchemaQuery {
   public static final String STMT_FETCH_INPUT =
       "SELECT " + COLUMN_SQI_ID + ", " + COLUMN_SQI_NAME + ", "
       + COLUMN_SQI_FORM + ", " + COLUMN_SQI_INDEX + ", " + COLUMN_SQI_TYPE
-      + ", " + COLUMN_SQI_STRMASK + ", " + COLUMN_SQI_STRLENGTH + " FROM "
-      + TABLE_SQ_INPUT + " WHERE " + COLUMN_SQI_FORM + " = ? ORDER BY "
+      + ", " + COLUMN_SQI_STRMASK + ", " + COLUMN_SQI_STRLENGTH
+      + ", cast(null as varchar(100)) FROM " + TABLE_SQ_INPUT + " WHERE "
+      + COLUMN_SQI_FORM + " = ? ORDER BY " + COLUMN_SQI_INDEX;
+
+  // DML: Fetch inputs and values for a given connection
+  public static final String STMT_FETCH_CONNECTION_INPUT =
+      "SELECT " + COLUMN_SQI_ID + ", " + COLUMN_SQI_NAME + ", "
+      + COLUMN_SQI_FORM + ", " + COLUMN_SQI_INDEX + ", " + COLUMN_SQI_TYPE
+      + ", " + COLUMN_SQI_STRMASK + ", " + COLUMN_SQI_STRLENGTH
+      + ", " + COLUMN_SQNI_VALUE + " FROM " + TABLE_SQ_INPUT
+      + " LEFT OUTER JOIN " + TABLE_SQ_CONNECTION_INPUT + " ON "
+      + COLUMN_SQNI_INPUT + " = " + COLUMN_SQI_ID + " WHERE "
+      + COLUMN_SQI_FORM + " = ? AND (" + COLUMN_SQNI_CONNECTION
+      + " = ? OR " + COLUMN_SQNI_CONNECTION + " IS NULL) ORDER BY "
       + COLUMN_SQI_INDEX;
 
   // DML: Insert connector base
@@ -232,6 +244,52 @@ public final class DerbySchemaQuery {
       + ", " + COLUMN_SQI_STRMASK + ", " + COLUMN_SQI_STRLENGTH + ") "
       + "VALUES (?, ?, ?, ?, ?, ?)";
 
+  // DML: Insert new connection
+  public static final String STMT_INSERT_CONNECTION =
+    "INSERT INTO " + TABLE_SQ_CONNECTION + " (" + COLUMN_SQN_NAME + ", "
+    + COLUMN_SQN_CONNECTOR + ") VALUES (?, ?)";
+
+  // DML: Insert new connection inputs
+  public static final String STMT_INSERT_CONNECTION_INPUT =
+    "INSERT INTO " + TABLE_SQ_CONNECTION_INPUT + " (" + COLUMN_SQNI_CONNECTION
+    + ", " + COLUMN_SQNI_INPUT + ", " + COLUMN_SQNI_VALUE + ") "
+    + "VALUES (?, ?, ?)";
+
+  // DML: Delete rows from connection input table
+  public static final String STMT_DELETE_CONNECTION_INPUT =
+    "DELETE FROM " + TABLE_SQ_CONNECTION_INPUT + " WHERE "
+    + COLUMN_SQNI_CONNECTION + " = ?";
+
+  // DML: Delete row from connection table
+  public static final String STMT_DELETE_CONNECTION =
+    "DELETE FROM " + TABLE_SQ_CONNECTION + " WHERE " + COLUMN_SQN_ID + " = ?";
+
+  // DML: Select one specific connection
+  public static final String STMT_SELECT_CONNECTION_SINGLE =
+    "SELECT " + COLUMN_SQN_ID + ", " + COLUMN_SQN_NAME + ", "
+    + COLUMN_SQN_CONNECTOR + " FROM " + TABLE_SQ_CONNECTION + " WHERE "
+    + COLUMN_SQN_ID + " = ?";
+
+  // DML: Select one specific connection
+  public static final String STMT_SELECT_CONNECTION_ALL =
+    "SELECT " + COLUMN_SQN_ID + ", " + COLUMN_SQN_NAME + ", "
+      + COLUMN_SQN_CONNECTOR + " FROM " + TABLE_SQ_CONNECTION;
+
+  // DML: Select all inputs for given connection
+  public static final String STMT_SELECT_CONNECTION_INPUT =
+    "SELECT " + COLUMN_SQF_ID + ", " + COLUMN_SQI_ID + ", " + COLUMN_SQNI_VALUE
+    + " FROM " + TABLE_SQ_CONNECTION_INPUT + " JOIN " + TABLE_SQ_INPUT
+    + " JOIN " + TABLE_SQ_FORM +  " ON " + COLUMN_SQF_ID + " = "
+    + COLUMN_SQI_FORM + " ON " + COLUMN_SQI_ID + " = " + COLUMN_SQNI_INPUT
+    + " WHERE " + COLUMN_SQNI_CONNECTION + " = ? ORDER BY "
+    + COLUMN_SQF_CONNECTOR + ", " + COLUMN_SQF_INDEX
+    + ", " + COLUMN_SQI_INDEX ;
+
+  // DML: Check if given connection exists
+  public static final String STMT_SELECT_CONNECTION_CHECK =
+    "SELECT count(*) FROM " + TABLE_SQ_CONNECTION + " WHERE " + COLUMN_SQN_ID
+    + " = ?";
+
   private DerbySchemaQuery() {
     // Disable explicit object creation
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java
new file mode 100644
index 0000000..49c93b4
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java
@@ -0,0 +1,208 @@
+/**
+ * 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.sqoop.handler;
+
+import org.apache.log4j.Logger;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.connector.ConnectorManager;
+import org.apache.sqoop.framework.FrameworkManager;
+import org.apache.sqoop.json.ConnectionBean;
+import org.apache.sqoop.json.JsonBean;
+import org.apache.sqoop.json.ValidationBean;
+import org.apache.sqoop.model.MConnection;
+import org.apache.sqoop.model.MConnectionForms;
+import org.apache.sqoop.repository.Repository;
+import org.apache.sqoop.repository.RepositoryManager;
+import org.apache.sqoop.server.RequestContext;
+import org.apache.sqoop.server.RequestHandler;
+import org.apache.sqoop.server.common.ServerError;
+import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.Validator;
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Connection request handler is supporting following resources:
+ *
+ * GET /v1/connection
+ * Get brief list of all connections present in the system.
+ *
+ * GET /v1/connection/:xid
+ * Return details about one particular connection with id :xid or about all of
+ * them if :xid equals to "all".
+ *
+ * POST /v1/connection
+ * Create new connection
+ *
+ * PUT /v1/connection/:xid
+ * Update connection with id :xid.
+ *
+ * DELETE /v1/connection/:xid
+ * Remove connection with id :xid
+ */
+public class ConnectionRequestHandler implements RequestHandler {
+
+  private static final Logger LOG =
+      Logger.getLogger(ConnectorRequestHandler.class);
+
+  public ConnectionRequestHandler() {
+    LOG.info("ConnectionRequestHandler initialized");
+  }
+
+  @Override
+  public JsonBean handleEvent(RequestContext ctx) throws SqoopException {
+    switch (ctx.getMethod()) {
+      case GET:
+        return getConnections(ctx);
+      case POST:
+        return createUpdateConnection(ctx, false);
+      case PUT:
+        return createUpdateConnection(ctx, true);
+      case DELETE:
+        return deleteConnection(ctx);
+    }
+
+    return null;
+  }
+
+  /**
+   * Delete connection from metadata repository.
+   *
+   * @param ctx Context object
+   * @return Empty bean
+   */
+  private JsonBean deleteConnection(RequestContext ctx) {
+    String sxid = ctx.getLastURLElement();
+    long xid = Long.valueOf(sxid);
+
+    Repository repository = RepositoryManager.getRepository();
+    repository.deleteConnection(xid);
+
+    return JsonBean.EMPTY_BEAN;
+  }
+
+  /**
+   * Update or create connection metadata in repository.
+   *
+   * @param ctx Context object
+   * @return Validation bean object
+   */
+  private JsonBean createUpdateConnection(RequestContext ctx, boolean update) {
+//    Check that given ID equals with sent ID, otherwise report an error UPDATE
+//    String sxid = ctx.getLastURLElement();
+//    long xid = Long.valueOf(sxid);
+
+    ConnectionBean bean = new ConnectionBean();
+
+    try {
+      JSONObject json =
+        (JSONObject) JSONValue.parse(ctx.getRequest().getReader());
+      bean.restore(json);
+    } catch (IOException e) {
+      throw new SqoopException(ServerError.SERVER_0003,
+        "Can't read request content", e);
+    }
+
+    // Get connection object
+    List<MConnection> connections = bean.getConnections();
+
+    if(connections.size() != 1) {
+      throw new SqoopException(ServerError.SERVER_0003,
+        "Expected one connection metadata but got " + connections.size());
+    }
+
+    MConnection connection = connections.get(0);
+
+    // Verify that user is not trying to spoof us
+    MConnectionForms connectorForms
+      = ConnectorManager.getConnectorMetadata(connection.getConnectorId())
+      .getConnectionForms();
+    MConnectionForms frameworkForms = FrameworkManager.getFramework()
+      .getConnectionForms();
+
+    if(!connectorForms.equals(connection.getConnectorPart())
+      || !frameworkForms.equals(connection.getFrameworkPart())) {
+      throw new SqoopException(ServerError.SERVER_0003,
+        "Detected incorrect form structure");
+    }
+
+    // Get validator objects
+    Validator connectorValidator =
+      ConnectorManager.getConnector(connection.getConnectorId()).getValidator();
+    Validator frameworkValidator = FrameworkManager.getValidator();
+
+    // Validate connection object
+    Status conStat = connectorValidator.validate(connection.getConnectorPart());
+    Status frmStat = frameworkValidator.validate(connection.getFrameworkPart());
+    Status finalStatus = Status.getWorstStatus(conStat, frmStat);
+
+    // If we're good enough let's perform the action
+    if(finalStatus.canProceed()) {
+      if(update) {
+        RepositoryManager.getRepository().updateConnection(connection);
+      } else {
+        RepositoryManager.getRepository().createConnection(connection);
+      }
+    }
+
+    // Return back validations in all cases
+    return new ValidationBean(connection, finalStatus);
+  }
+
+  private JsonBean getConnections(RequestContext ctx) {
+    String sxid = ctx.getLastURLElement();
+    ConnectionBean bean;
+
+    Locale locale = ctx.getAcceptLanguageHeader();
+    Repository repository = RepositoryManager.getRepository();
+
+    if (sxid.equals("all")) {
+
+      List<MConnection> connections = repository.findConnections();
+      bean = new ConnectionBean(connections);
+
+      // Add associated resources into the bean
+      for( MConnection connection : connections) {
+        long connectorId = connection.getConnectorId();
+        if(!bean.hasConnectorBundle(connectorId)) {
+          bean.addConnectorBundle(connectorId,
+            ConnectorManager.getResourceBundle(connectorId, locale));
+        }
+      }
+    } else {
+      long xid = Long.valueOf(sxid);
+
+      MConnection connection = repository.findConnection(xid);
+      long connectorId = connection.getConnectorId();
+
+      bean = new ConnectionBean(connection);
+
+      bean.addConnectorBundle(connectorId,
+        ConnectorManager.getResourceBundle(connectorId, locale));
+    }
+
+    // Sent framework resource bundle in all cases
+    bean.setFrameworkBundle(FrameworkManager.getBundle(locale));
+
+    return bean;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/handler/ConnectorRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/ConnectorRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/ConnectorRequestHandler.java
index e5eb444..b8dc3b8 100644
--- a/server/src/main/java/org/apache/sqoop/handler/ConnectorRequestHandler.java
+++ b/server/src/main/java/org/apache/sqoop/handler/ConnectorRequestHandler.java
@@ -19,7 +19,6 @@ package org.apache.sqoop.handler;
 
 import org.apache.log4j.Logger;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.connector.ConnectorHandler;
 import org.apache.sqoop.connector.ConnectorManager;
 import org.apache.sqoop.json.JsonBean;
 import org.apache.sqoop.json.ConnectorBean;
@@ -27,6 +26,11 @@ import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.server.RequestContext;
 import org.apache.sqoop.server.RequestHandler;
 
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.ResourceBundle;
+
 public class ConnectorRequestHandler implements RequestHandler {
 
   private static final Logger LOG =
@@ -39,22 +43,28 @@ public class ConnectorRequestHandler implements RequestHandler {
 
   @Override
   public JsonBean handleEvent(RequestContext ctx) throws SqoopException {
-    MConnector[] connectors;
+    List<MConnector> connectors;
+    List<ResourceBundle> bundles;
+    Locale locale = ctx.getAcceptLanguageHeader();
+
+    String cid = ctx.getLastURLElement();
 
-    String uri = ctx.getRequest().getRequestURI();
-    int slash = uri.lastIndexOf("/");
-    String cid = uri.substring(slash + 1);
     LOG.info("ConnectorRequestHandler handles cid: " + cid);
     if (cid.equals("all")) {
       // display all connectors
-      connectors = ConnectorManager.getConnectors();
+      connectors = ConnectorManager.getConnectorsMetadata();
+      bundles = ConnectorManager.getResourceBundles(locale);
 
     } else {
-      // display one connector
-      connectors = new MConnector[] {
-          ConnectorManager.getConnector(Long.parseLong(cid)) };
+      Long id = Long.parseLong(cid);
+
+      connectors = new LinkedList<MConnector>();
+      bundles = new LinkedList<ResourceBundle>();
+
+      connectors.add(ConnectorManager.getConnectorMetadata(id));
+      bundles.add(ConnectorManager.getResourceBundle(id, locale));
     }
 
-    return new ConnectorBean(connectors);
+    return new ConnectorBean(connectors, bundles);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
index 1041e71..9a60c63 100644
--- a/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
+++ b/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
@@ -28,7 +28,7 @@ import org.apache.sqoop.server.RequestHandler;
 /**
  *
  */
-public class FrameworkRequestHandler implements RequestHandler {
+public class FrameworkRequestHandler  implements RequestHandler {
 
   private static final Logger LOG =
       Logger.getLogger(ConnectorRequestHandler.class);
@@ -39,6 +39,7 @@ public class FrameworkRequestHandler implements RequestHandler {
 
   @Override
   public JsonBean handleEvent(RequestContext ctx) throws SqoopException {
-    return new FrameworkBean(FrameworkManager.getFramework());
+    return new FrameworkBean(FrameworkManager.getFramework(),
+      FrameworkManager.getBundle(ctx.getAcceptLanguageHeader()));
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/server/RequestContext.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/RequestContext.java b/server/src/main/java/org/apache/sqoop/server/RequestContext.java
index b1ea853..90e174e 100644
--- a/server/src/main/java/org/apache/sqoop/server/RequestContext.java
+++ b/server/src/main/java/org/apache/sqoop/server/RequestContext.java
@@ -17,11 +17,25 @@
  */
 package org.apache.sqoop.server;
 
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.server.common.ServerError;
+
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import java.util.Locale;
 
 public class RequestContext {
 
+  /**
+   * Enumeration with supported HTTP methods.
+   */
+  public enum Method {
+    GET,
+    POST,
+    PUT,
+    DELETE,
+  }
+
   private final HttpServletRequest request;
   private final HttpServletResponse response;
 
@@ -41,4 +55,41 @@ public class RequestContext {
   public String getPath() {
     return request.getRequestURL().toString();
   }
+
+  /**
+   * Get method that was used for this HTTP request.
+   *
+   * @return Method that was used
+   */
+  public Method getMethod() {
+    try {
+      return Method.valueOf(request.getMethod());
+    } catch(IllegalArgumentException ex) {
+      throw new SqoopException(ServerError.SERVER_0002,
+        "Unsupported HTTP method:" + request.getMethod());
+    }
+  }
+
+  /**
+   * Return last element of URL.
+   *
+   * Return text occurring after last "/" character in URL, typically there will
+   * be an ID.
+   *
+   * @return String after last "/" in URL
+   */
+  public String getLastURLElement() {
+    String uri = getRequest().getRequestURI();
+    int slash = uri.lastIndexOf("/");
+    return uri.substring(slash + 1);
+  }
+
+  /**
+   * Get locale specified in accept-language HTTP header.
+   *
+   * @return First specified locale
+   */
+  public Locale getAcceptLanguageHeader() {
+    return new Locale(request.getHeader("Accept-Language"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java b/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java
index f36aa05..ece3d93 100644
--- a/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java
+++ b/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java
@@ -18,8 +18,6 @@
 package org.apache.sqoop.server;
 
 import java.io.IOException;
-import java.io.Writer;
-import java.nio.charset.Charset;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
@@ -28,7 +26,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.log4j.Logger;
 import org.apache.sqoop.common.ErrorCode;
-import org.apache.sqoop.common.ExceptionInfo;
+import org.apache.sqoop.json.ExceptionInfo;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.common.SqoopProtocolConstants;
 import org.apache.sqoop.common.SqoopResponseCode;
@@ -72,6 +70,38 @@ public class SqoopProtocolServlet extends HttpServlet {
     }
   }
 
+  @Override
+  protected final void doPut(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    RequestContext rctx = new RequestContext(req, resp);
+
+    try {
+      JsonBean bean = handlePutRequest(rctx);
+      if (bean != null) {
+        sendSuccessResponse(rctx, bean);
+      }
+    } catch (Exception ex) {
+      LOG.error("Exception in PUT " + rctx.getPath(), ex);
+      sendErrorResponse(rctx, ex);
+    }
+  }
+
+  @Override
+  protected final void doDelete(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    RequestContext rctx = new RequestContext(req, resp);
+
+    try {
+      JsonBean bean = handleDeleteRequest(rctx);
+      if (bean != null) {
+        sendSuccessResponse(rctx, bean);
+      }
+    } catch (Exception ex) {
+      LOG.error("Exception in DELETE " + rctx.getPath(), ex);
+      sendErrorResponse(rctx, ex);
+    }
+  }
+
   private void sendSuccessResponse(RequestContext ctx, JsonBean bean)
       throws IOException {
     HttpServletResponse response = ctx.getResponse();
@@ -106,9 +136,11 @@ public class SqoopProtocolServlet extends HttpServlet {
           SqoopProtocolConstants.HEADER_SQOOP_INTERNAL_ERROR_MESSAGE,
           ex.getMessage());
 
-      response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR,
-          new ExceptionInfo(ec.getCode(),
-              ex.getMessage(), ex).extract().toJSONString());
+      ExceptionInfo exceptionInfo = new ExceptionInfo(ec.getCode(),
+        ex.getMessage(), ex);
+
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      response.getWriter().write(exceptionInfo.extract().toJSONString());
     } else {
       response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
     }
@@ -125,8 +157,6 @@ public class SqoopProtocolServlet extends HttpServlet {
         code.getCode());
     response.setHeader(SqoopProtocolConstants.HEADER_SQOOP_ERROR_MESSAGE,
         code.getMessage());
-
-
   }
 
   protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
@@ -140,4 +170,16 @@ public class SqoopProtocolServlet extends HttpServlet {
 
     return null;
   }
+
+  protected JsonBean handlePutRequest(RequestContext ctx) throws Exception {
+    super.doPut(ctx.getRequest(), ctx.getResponse());
+
+    return null;
+  }
+
+  protected JsonBean handleDeleteRequest(RequestContext ctx) throws Exception {
+    super.doDelete(ctx.getRequest(), ctx.getResponse());
+
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/common/ServerError.java b/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
new file mode 100644
index 0000000..0a97d2d
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/server/common/ServerError.java
@@ -0,0 +1,51 @@
+/**
+ * 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.sqoop.server.common;
+
+import org.apache.sqoop.common.ErrorCode;
+
+/**
+ *
+ */
+public enum ServerError implements ErrorCode {
+
+  /** Unknown error on server side. */
+  SERVER_0001("Unknown server error"),
+
+  /** Unknown error on server side. */
+  SERVER_0002("Unsupported HTTP method"),
+
+  /** We've received invalid HTTP request */
+  SERVER_0003("Invalid HTTP request"),
+
+  ;
+
+  private final String message;
+
+  private ServerError(String message) {
+    this.message = message;
+  }
+
+  public String getCode() {
+    return name();
+  }
+
+  public String getMessage() {
+    return message;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java
new file mode 100644
index 0000000..e8af4d7
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java
@@ -0,0 +1,56 @@
+/**
+ * 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.sqoop.server.v1;
+
+import org.apache.sqoop.handler.ConnectionRequestHandler;
+import org.apache.sqoop.json.JsonBean;
+import org.apache.sqoop.server.RequestContext;
+import org.apache.sqoop.server.RequestHandler;
+import org.apache.sqoop.server.SqoopProtocolServlet;
+
+/**
+ *
+ */
+public class ConnectionServlet extends SqoopProtocolServlet {
+
+  private RequestHandler requestHandler;
+
+  public ConnectionServlet() {
+    requestHandler = new ConnectionRequestHandler();
+  }
+
+  @Override
+  protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
+    return requestHandler.handleEvent(ctx);
+  }
+
+  @Override
+  protected JsonBean handlePostRequest(RequestContext ctx) throws Exception {
+    return requestHandler.handleEvent(ctx);
+  }
+
+  @Override
+  protected JsonBean handlePutRequest(RequestContext ctx) throws Exception {
+    return requestHandler.handleEvent(ctx);
+  }
+
+  @Override
+  protected JsonBean handleDeleteRequest(RequestContext ctx) throws Exception {
+    return requestHandler.handleEvent(ctx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/server/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/server/src/main/webapp/WEB-INF/web.xml b/server/src/main/webapp/WEB-INF/web.xml
index dfd717f..f0af9ce 100644
--- a/server/src/main/webapp/WEB-INF/web.xml
+++ b/server/src/main/webapp/WEB-INF/web.xml
@@ -63,5 +63,17 @@ limitations under the License.
     <url-pattern>/v1/framework/*</url-pattern>
   </servlet-mapping>
 
+  <!-- Connection servlet -->
+  <servlet>
+    <servlet-name>v1.ConnectionServlet</servlet-name>
+    <servlet-class>org.apache.sqoop.server.v1.ConnectionServlet</servlet-class>
+    <load-on-startup>1</load-on-startup>
+  </servlet>
+
+  <servlet-mapping>
+    <servlet-name>v1.ConnectionServlet</servlet-name>
+    <url-pattern>/v1/connection/*</url-pattern>
+  </servlet-mapping>
+
 </web-app>
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/spi/src/main/java/org/apache/sqoop/validation/Status.java
----------------------------------------------------------------------
diff --git a/spi/src/main/java/org/apache/sqoop/validation/Status.java b/spi/src/main/java/org/apache/sqoop/validation/Status.java
deleted file mode 100644
index 2c96e4b..0000000
--- a/spi/src/main/java/org/apache/sqoop/validation/Status.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sqoop.validation;
-
-/**
- * Status modes of a validation process.
- */
-public enum Status {
-  /**
-   * Everything is correct.
-   *
-   * There are no issues, no warnings, nothing.
-   */
-  FINE,
-
-  /**
-   * Validated entity is correct enough to be processed.
-   *
-   * There might be some warnings, but no errors. It should be safe
-   * to proceed with processing.
-   */
-  ACCEPTABLE,
-
-  /**
-   * There are serious issues with validated entity.
-   *
-   * We can't proceed until reported issues will be resolved.
-   */
-  UNACCEPTABLE,
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/47cb311a/spi/src/main/java/org/apache/sqoop/validation/Validator.java
----------------------------------------------------------------------
diff --git a/spi/src/main/java/org/apache/sqoop/validation/Validator.java b/spi/src/main/java/org/apache/sqoop/validation/Validator.java
index 4da5e4a..b411ffc 100644
--- a/spi/src/main/java/org/apache/sqoop/validation/Validator.java
+++ b/spi/src/main/java/org/apache/sqoop/validation/Validator.java
@@ -19,7 +19,6 @@ package org.apache.sqoop.validation;
 
 import org.apache.sqoop.model.MConnectionForms;
 import org.apache.sqoop.model.MForm;
-import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MJobForms;
 
 import java.util.List;
@@ -71,9 +70,7 @@ public class Validator {
     for (MForm form : forms) {
       Status status = validate(form);
 
-      if ( finalStatus.compareTo(status) > 0 ) {
-        finalStatus = status;
-      }
+      finalStatus = Status.getWorstStatus(finalStatus, status);
     }
 
     return finalStatus;