commit b71b4caccdea3895ac9e50b89ca5ee978ffb01b1 Author: Daniel Shahaf Date: Wed Mar 28 00:04:41 2012 +0000 move sqoop to TLP git-svn-id: https://svn.apache.org/repos/asf/sqoop/branches/sqoop2@1306066 13f79535-47bb-0310-9956-ffa450edef68 diff --git a/.gitignore b/.gitignore new file mode 100644 index 00000000..beef00d0 --- /dev/null +++ b/.gitignore @@ -0,0 +1,4 @@ +.classpath +.project +.settings +target diff --git a/DISCLAIMER.txt b/DISCLAIMER.txt new file mode 100644 index 00000000..460e6652 --- /dev/null +++ b/DISCLAIMER.txt @@ -0,0 +1,16 @@ +Apache Sqoop is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator Project Management +Committee. + +Incubation is required for all newly accepted projects until a further review +indicates that the infrastructure, communications, and decisions making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness or +stability of the code, it does indicate that the project has yet to be fully +endorsed by the ASF. + +For more information about the incubation status of the Sqoop project you can +go to the following page: + +http://incubator.apache.org/projects/sqoop.html diff --git a/LICENSE.txt b/LICENSE.txt new file mode 100644 index 00000000..d6456956 --- /dev/null +++ b/LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License 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. diff --git a/NOTICE.txt b/NOTICE.txt new file mode 100644 index 00000000..cb22834f --- /dev/null +++ b/NOTICE.txt @@ -0,0 +1,8 @@ +Apache Sqoop (incubating) +Copyright 2011-2012 The Apache Software Foundation + +This product includes softare developed at +The Apache Software Foundation (http://www.apache.org/). + +Portions of this software were developed at +Cloudera, Inc. (http://www.cloudera.com/). diff --git a/README.txt b/README.txt new file mode 100644 index 00000000..80d421ec --- /dev/null +++ b/README.txt @@ -0,0 +1,13 @@ +---- +The sqoop2 branch is an experimental branch for the development of the +next major release of Sqoop. + +For more details please see the following JIRA issue: +https://issues.apache.org/jira/browse/SQOOP-365 +---- + +Welcome to Sqoop! + +This document outlines how to build the source code. + +TODO: add details diff --git a/client/pom.xml b/client/pom.xml new file mode 100644 index 00000000..fb7e11c6 --- /dev/null +++ b/client/pom.xml @@ -0,0 +1,46 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-client + Sqoop Client + + + + org.apache.sqoop + sqoop-core + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-common + 2.0.0-SNAPSHOT + + + diff --git a/common/pom.xml b/common/pom.xml new file mode 100644 index 00000000..2ab9073c --- /dev/null +++ b/common/pom.xml @@ -0,0 +1,101 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-common + Sqoop Common + + + + com.googlecode.json-simple + json-simple + + + junit + junit + test + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + save-version + generate-sources + + run + + + + + + + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-source + generate-sources + + add-source + + + + ${project.build.directory}/generated-sources/java + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/common/src/main/java/org/apache/sqoop/common/ErrorCode.java b/common/src/main/java/org/apache/sqoop/common/ErrorCode.java new file mode 100644 index 00000000..f53d9729 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/ErrorCode.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.common; + +/** + * Defines an error-code contract. Sqoop exceptions use the error code to + * communicate error information where possible. Each error code is associated + * with default message that identifies the high level information related to + * the underlying error condition. + */ +public interface ErrorCode { + + /** + * @return the string representation of the error code. + */ + public String getCode(); + + /** + * @return the message associated with error code. + */ + public String getMessage(); +} diff --git a/common/src/main/java/org/apache/sqoop/common/ExceptionInfo.java b/common/src/main/java/org/apache/sqoop/common/ExceptionInfo.java new file mode 100644 index 00000000..b55111fe --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/ExceptionInfo.java @@ -0,0 +1,64 @@ +/** + * 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.common; + +import java.io.PrintWriter; +import java.io.StringWriter; + +import org.apache.sqoop.json.JsonBean; +import org.json.simple.JSONObject; + +public class ExceptionInfo implements JsonBean { + + public static final String ERROR_CODE = "error-code"; + public static final String ERROR_MESSAGE = "error-message"; + public static final String STACK_TRACE = "stack-trace"; + + private String errorCode; + private String errorMessage; + private String stackTrace; + + public ExceptionInfo(String code, String message, Exception ex) { + errorCode = code; + errorMessage = message; + + StringWriter writer = new StringWriter(); + ex.printStackTrace(new PrintWriter(writer)); + writer.flush(); + + stackTrace = writer.getBuffer().toString(); + } + + @SuppressWarnings("unchecked") + @Override + public JSONObject extract() { + JSONObject result = new JSONObject(); + result.put(ERROR_CODE, errorCode); + result.put(ERROR_MESSAGE, errorMessage); + result.put(STACK_TRACE, stackTrace); + + return result; + } + + @Override + public void restore(JSONObject jsonObject) { + errorCode = (String) jsonObject.get(ERROR_CODE); + errorMessage = (String) jsonObject.get(ERROR_MESSAGE); + stackTrace = (String) jsonObject.get(STACK_TRACE); + } +} diff --git a/common/src/main/java/org/apache/sqoop/common/SqoopException.java b/common/src/main/java/org/apache/sqoop/common/SqoopException.java new file mode 100644 index 00000000..6caed13a --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/SqoopException.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.common; + + +/** + * Base exception for Sqoop framework. This exception requires the specification + * of an error code for reference purposes. Where necessary the appropriate + * constructor can be used to pass in additional message beyond what is + * specified by the error code and/or the causal exception. + */ +@SuppressWarnings("serial") +public class SqoopException extends RuntimeException { + + private final ErrorCode code; + + public SqoopException(ErrorCode code) { + super(code.getCode() + ":" + code.getMessage()); + this.code = code; + } + + public SqoopException(ErrorCode code, String extraInfo) { + super(code.getCode() + ":" + code.getMessage() + " - " + extraInfo); + this.code = code; + } + + public SqoopException(ErrorCode code, Throwable cause) { + super(code.getCode() + ":" + code.getMessage(), cause); + this.code = code; + } + + public SqoopException(ErrorCode code, String extraInfo, Throwable cause) { + super(code.getCode() + ":" + code.getMessage() + " - " + extraInfo, cause); + this.code = code; + } + + public ErrorCode getErrorCode() { + return code; + } +} diff --git a/common/src/main/java/org/apache/sqoop/common/SqoopProtocolConstants.java b/common/src/main/java/org/apache/sqoop/common/SqoopProtocolConstants.java new file mode 100644 index 00000000..b06e0595 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/SqoopProtocolConstants.java @@ -0,0 +1,38 @@ +/** + * 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.common; + +public class SqoopProtocolConstants { + + public static final String HEADER_SQOOP_ERROR_CODE = "sqoop-error-code"; + + public static final String HEADER_SQOOP_ERROR_MESSAGE = "sqoop-error-message"; + + public static final String HEADER_SQOOP_INTERNAL_ERROR_CODE = + "sqoop-internal-error-code"; + + public static final String HEADER_SQOOP_INTERNAL_ERROR_MESSAGE = + "sqoop-internal-error-message"; + + public static final String JSON_CONTENT_TYPE = + "application/json; charset=\"UTF-8\""; + + private SqoopProtocolConstants() { + // Disable explicit object creation + } +} diff --git a/common/src/main/java/org/apache/sqoop/common/SqoopResponseCode.java b/common/src/main/java/org/apache/sqoop/common/SqoopResponseCode.java new file mode 100644 index 00000000..93a43f5d --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/SqoopResponseCode.java @@ -0,0 +1,45 @@ +/** + * 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.common; + +public enum SqoopResponseCode { + + SQOOP_1000("1000", "OK"), + SQOOP_2000("2000", "ERROR"); + + private final String code; + private final String message; + + private SqoopResponseCode(String code, String message) { + this.code = code; + this.message = message; + } + + public String getCode() { + return code; + } + + public String getMessage() { + return message; + } + + public static SqoopResponseCode getFromCode(String code) { + return SqoopResponseCode.valueOf("SQOOP_" + code); + } + +} diff --git a/common/src/main/java/org/apache/sqoop/common/VersionAnnotation.java b/common/src/main/java/org/apache/sqoop/common/VersionAnnotation.java new file mode 100644 index 00000000..acd901f1 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/VersionAnnotation.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.common; + +import java.lang.annotation.*; + +/** + * A package attribute that captures the version that was compiled. + * Copied down from hadoop. All is same except name of interface. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.PACKAGE) +public @interface VersionAnnotation { + + /** + * Get the Hadoop version + * @return the version string (eg. "2.0.0") + */ + String version(); + + /** + * Get the username that compiled the code. + */ + String user(); + + /** + * Get the date when the code was compiled. + * @return the date in unix 'date' format + */ + String date(); + + /** + * Get the url for the subversion repository. + */ + String url(); + + /** + * Get the subversion revision. + * @return the revision number as a string (eg. "451451") + */ + String revision(); +} diff --git a/common/src/main/java/org/apache/sqoop/common/VersionInfo.java b/common/src/main/java/org/apache/sqoop/common/VersionInfo.java new file mode 100644 index 00000000..02f8cf3e --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/common/VersionInfo.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.sqoop.common; + +public class VersionInfo { + + private static Package myPackage; + private static VersionAnnotation annotation; + + static { + myPackage = VersionAnnotation.class.getPackage(); + annotation = myPackage.getAnnotation(VersionAnnotation.class); + } + + private VersionInfo() { + // Disable explicit object creation + } + + /** + * Get the version. + * @return the version string, eg. "2.0.0" + */ + public static String getVersion() { + return annotation != null ? annotation.version() : "Unknown"; + } + + /** + * Get the subversion revision number for the root directory + * @return the revision number, eg. "451451" + */ + public static String getRevision() { + return annotation != null ? annotation.revision() : "Unknown"; + } + + /** + * The date that the code was compiled. + * @return the compilation date in unix date format + */ + public static String getDate() { + return annotation != null ? annotation.date() : "Unknown"; + } + + /** + * The user that compiled the code. + * @return the username of the user + */ + public static String getUser() { + return annotation != null ? annotation.user() : "Unknown"; + } + + /** + * Get the subversion URL for the root directory. + * @return the url + */ + public static String getUrl() { + return annotation != null ? annotation.url() : "Unknown"; + } + +} diff --git a/common/src/main/java/org/apache/sqoop/json/JsonBean.java b/common/src/main/java/org/apache/sqoop/json/JsonBean.java new file mode 100644 index 00000000..aa84935a --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/json/JsonBean.java @@ -0,0 +1,39 @@ +/** + * 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.json; + +import org.json.simple.JSONObject; + +public interface JsonBean { + + public JSONObject extract(); + + public void restore(JSONObject jsonObject); + + public static final JsonBean EMPTY_BEAN = new JsonBean() { + + @Override + public JSONObject extract() { + return new JSONObject(); + } + + @Override + public void restore(JSONObject jsonObject) { + } + }; +} diff --git a/common/src/main/java/org/apache/sqoop/json/VersionBean.java b/common/src/main/java/org/apache/sqoop/json/VersionBean.java new file mode 100644 index 00000000..03e8c0b8 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/json/VersionBean.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.json; + +import java.util.Arrays; + +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; + +public class VersionBean implements JsonBean { + + public static final String VERSION = "version"; + public static final String REVISION = "revision"; + public static final String DATE = "date"; + public static final String USER = "user"; + public static final String URL = "url"; + public static final String PROTOCOLS = "protocols"; + + private String version; + private String revision; + private String date; + private String user; + private String url; + private String[] protocols; + + // for "extract" + public VersionBean(String version, String revision, String date, + String user, String url, String[] protocols) { + this.version = version; + this.revision = revision; + this.date = date; + this.user = user; + this.url = url; + this.protocols = new String[protocols.length]; + System.arraycopy(protocols, 0, this.protocols, 0, protocols.length); + } + + // for "restore" + public VersionBean() { + } + + @SuppressWarnings("unchecked") + @Override + public JSONObject extract() { + JSONObject result = new JSONObject(); + result.put(VERSION, version); + result.put(REVISION, revision); + result.put(DATE, date); + result.put(USER, user); + result.put(URL, url); + JSONArray protocolsArray = new JSONArray(); + for (String protocol : protocols) { + protocolsArray.add(protocol); + } + result.put(PROTOCOLS, protocolsArray); + return result; + } + + @Override + public void restore(JSONObject jsonObject) { + this.version = (String)jsonObject.get(VERSION); + this.revision = (String)jsonObject.get(REVISION); + this.date = (String)jsonObject.get(DATE); + this.user = (String)jsonObject.get(USER); + this.url = (String)jsonObject.get(URL); + JSONArray protocolsArray = (JSONArray) jsonObject.get(PROTOCOLS); + int size = protocolsArray.size(); + this.protocols = new String[size]; + for (int i = 0; i connectionForms; + private final List jobForms; + + public MConnector(String uniqueName, String className, + List connectionForms, List jobForms) { + if (uniqueName == null || className == null) { + throw new NullPointerException(); + } + + this.uniqueName = uniqueName; + this.className = className; + + this.connectionForms = new ArrayList(connectionForms.size()); + this.connectionForms.addAll(connectionForms); + + this.jobForms = new ArrayList(jobForms.size()); + this.jobForms.addAll(jobForms); + } + + public String getUniqueName() { + return uniqueName; + } + + public String getClassName() { + return className; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("connector-"); + sb.append(uniqueName).append(":").append(getPersistenceId()).append(":"); + sb.append(className).append("; conn-forms:").append(connectionForms); + sb.append("; job-forms:").append(jobForms); + + return sb.toString(); + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (!(other instanceof MConnector)) { + return false; + } + + MConnector mc = (MConnector) other; + return (uniqueName.equals(mc.uniqueName) + && className.equals(mc.className)) + && connectionForms.equals(mc.connectionForms) + && jobForms.equals(mc.jobForms); + } + + @Override + public int hashCode() { + int result = 23; + result = 31 * result + uniqueName.hashCode(); + result = 31 * result + className.hashCode(); + for (MForm cmf : connectionForms) { + result = 31 * result + cmf.hashCode(); + } + for (MForm jmf : jobForms) { + result = 31 * result + jmf.hashCode(); + } + + return result; + } + + public List getConnectionForms() { + return connectionForms; + } + + public List getJobForms() { + return jobForms; + } +} diff --git a/common/src/main/java/org/apache/sqoop/model/MForm.java b/common/src/main/java/org/apache/sqoop/model/MForm.java new file mode 100644 index 00000000..515bf5e0 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MForm.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.sqoop.model; + +import java.util.List; + +/** + * Represents a group of inputs that are processed together. This allows the + * input gathering process to be broken down into multiple steps that can be + * then paged through by the user interface. + */ +public final class MForm extends MNamedElement { + + private final List> inputs; + + public MForm(String name, List> inputs) { + super(name); + + this.inputs = inputs; + } + + public List> getInputs() { + return inputs; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("form-").append(getName()); + sb.append(":").append(getPersistenceId()).append(":").append(inputs); + + return sb.toString(); + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (!(other instanceof MForm)) { + return false; + } + + MForm mf = (MForm) other; + return getName().equals(mf.getName()) + && inputs.equals(mf.inputs); + } + + @Override + public int hashCode() { + int result = 17; + result = 31 * result + getName().hashCode(); + for (MInput mi : inputs) { + result = 31 * result + mi.hashCode(); + } + + return result; + } +} diff --git a/common/src/main/java/org/apache/sqoop/model/MFormType.java b/common/src/main/java/org/apache/sqoop/model/MFormType.java new file mode 100644 index 00000000..2f403df3 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MFormType.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.model; + +/** + * Represents the various form types supported by the system. + */ +public enum MFormType { + + /** Unknown form type */ + OTHER, + + /** Connection form type */ + CONNECTION, + + /** Job form type */ + JOB; + +} diff --git a/common/src/main/java/org/apache/sqoop/model/MInput.java b/common/src/main/java/org/apache/sqoop/model/MInput.java new file mode 100644 index 00000000..3171f8ca --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MInput.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.model; + +/** + * Represents a parameter input used by the connector for creating a connection + * or a job object. + * @param the value type associated with this parameter + */ +public abstract class MInput extends MNamedElement { + private T value; + + protected MInput(String name) { + super(name); + } + + /** + * @param value the value to be set for this parameter + */ + public void setValue(T value) { + this.value = value; + } + + /** + * @return any previously set value for this parameter + */ + public T getValue() { + return value; + } + + /** + * @return a URL-safe string representation of the value + */ + public abstract String getUrlSafeValueString(); + + /** + * Overrides the associated value of this input by the value represented by + * the provided URL-safe value string. + * @param valueString the string representation of the value from which the + * value must be restored. + */ + public abstract void restoreFromUrlSafeValueString(String valueString); + + public abstract MInputType getType(); + + /** + * @return true if this type maintains more state than what is + * stored in the MInput base class. + */ + protected abstract boolean hasExtraInfo(); + + /** + * @return the string representation of state stored in this type if + * applicable or an empty string. + */ + protected abstract String getExtraInfoToString(); + + /** + * All input types must override the equals() method such that the + * test for equality is based on static metadata only. As a result any + * set value, error message and other dynamic value data is not considered + * as part of the equality comparison. + */ + @Override + public abstract boolean equals(Object other); + + /** + * All input types must override the hashCode() method such that + * the hash code computation is solely based on static metadata. As a result + * any set value, error message and other dynamic value data is not + * considered as part of the hash code computation. + */ + @Override + public abstract int hashCode(); + + @Override + public final String toString() { + StringBuilder sb = new StringBuilder("input-").append(getName()); + sb.append(":").append(getPersistenceId()).append(":"); + sb.append(getType()); + if (hasExtraInfo()) { + sb.append(":").append(getExtraInfoToString()); + } + + return sb.toString(); + } +} diff --git a/common/src/main/java/org/apache/sqoop/model/MInputType.java b/common/src/main/java/org/apache/sqoop/model/MInputType.java new file mode 100644 index 00000000..7acdbca1 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MInputType.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.sqoop.model; + +/** + * Represents the various input types supported by the system. + */ +public enum MInputType { + + /** Unknown input type */ + OTHER, + + /** String input type */ + STRING, + + /** Map input type */ + MAP; +} diff --git a/common/src/main/java/org/apache/sqoop/model/MMapInput.java b/common/src/main/java/org/apache/sqoop/model/MMapInput.java new file mode 100644 index 00000000..528f6670 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MMapInput.java @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.model; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.sqoop.utils.UrlSafeUtils; + +public final class MMapInput extends MInput> { + + public MMapInput(String name) { + super(name); + } + + @Override + public String getUrlSafeValueString() { + Map valueMap = getValue(); + if (valueMap == null || valueMap.size() == 0) { + return ""; + } + boolean first = true; + StringBuilder vsb = new StringBuilder(); + for (String key : valueMap.keySet()) { + if (first) { + first = false; + } else { + vsb.append("&"); + } + String value = valueMap.get(key); + vsb.append(UrlSafeUtils.urlEncode(key)).append("="); + vsb.append(UrlSafeUtils.urlEncode(value)); + } + return vsb.toString(); + } + + @Override + public void restoreFromUrlSafeValueString(String valueString) { + Map valueMap = null; + if (valueString != null && valueString.trim().length() > 0) { + valueMap = new HashMap(); + String[] valuePairs = valueString.split("&"); + for (String pair : valuePairs) { + String[] nameAndVal = pair.split("="); + if (nameAndVal.length > 0) { + String name = nameAndVal[0]; + String value = null; + if (nameAndVal.length > 1) { + value = nameAndVal[1]; + } + + valueMap.put(name, value); + } + } + } + setValue(valueMap); + } + + @Override + public MInputType getType() { + return MInputType.MAP; + } + + @Override + protected boolean hasExtraInfo() { + return false; + } + + @Override + protected String getExtraInfoToString() { + return ""; + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (!(other instanceof MMapInput)) { + return false; + } + + MMapInput mmi = (MMapInput) other; + return getName().equals(mmi.getName()); + } + + @Override + public int hashCode() { + return 23 + 31 * getName().hashCode(); + } +} diff --git a/common/src/main/java/org/apache/sqoop/model/MNamedElement.java b/common/src/main/java/org/apache/sqoop/model/MNamedElement.java new file mode 100644 index 00000000..0f063f77 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MNamedElement.java @@ -0,0 +1,76 @@ +/** + * 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.model; + +/** + * Represents an element of metadata used by the connector. + */ +public abstract class MNamedElement extends MPersistableEntity { + private static final String LABEL_KEY_SUFFIX = "-label"; + private static final String HELP_KEY_SUFFIX = "-help"; + + private final String name; + private final String labelKey; + private final String helpKey; + + private String errorMessage; + + protected MNamedElement(String name) { + this.name = name; + + labelKey = name + LABEL_KEY_SUFFIX; + helpKey = name + HELP_KEY_SUFFIX; + } + + /** + * @return the name of this parameter + */ + public String getName() { + return name; + } + + /** + * @return the label key to be used for this parameter + */ + public String getLabelKey() { + return labelKey; + } + + /** + * @return the help key to be used for this parameter + */ + public String getHelpKey() { + return helpKey; + } + + /** + * @param errMsg any error message associated with this parameter + */ + public void setErrorMessage(String errMsg) { + this.errorMessage = errMsg; + } + + /** + * @return any error message associated with this parameter + */ + public String getErrorMessage() { + return this.errorMessage; + } + + public abstract String toString(); +} diff --git a/common/src/main/java/org/apache/sqoop/model/MPersistableEntity.java b/common/src/main/java/org/apache/sqoop/model/MPersistableEntity.java new file mode 100644 index 00000000..880f54fc --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MPersistableEntity.java @@ -0,0 +1,41 @@ +/** + * 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.model; + +/** + * Represents a persistable metadata entity. + */ +public abstract class MPersistableEntity { + + private long persistenceId = -1L; + + protected MPersistableEntity() { + // Default constructor + } + + public void setPersistenceId(long persistenceId) { + this.persistenceId = persistenceId; + } + + public long getPersistenceId() { + return persistenceId; + } + + @Override + public abstract String toString(); +} diff --git a/common/src/main/java/org/apache/sqoop/model/MStringInput.java b/common/src/main/java/org/apache/sqoop/model/MStringInput.java new file mode 100644 index 00000000..37be3f81 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/model/MStringInput.java @@ -0,0 +1,108 @@ +/** + * 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.model; + +import org.apache.sqoop.utils.UrlSafeUtils; + +/** + * Represents a String input. The boolean flag mask supplied + * to its constructor can be used to indicate if the string should be masked + * from user-view. This is helpful for creating input strings that represent + * sensitive information such as passwords. + */ +public final class MStringInput extends MInput { + + private final boolean mask; + private final short maxLength; + + /** + * @param name the parameter name + * @param label the parameter label + * @param mask a flag indicating if the string should be masked + * @param maxLength the maximum length of the string + */ + public MStringInput(String name, boolean mask, short maxLength) { + super(name); + this.mask = mask; + this.maxLength = maxLength; + } + + /** + * @return true if this string represents sensitive information that + * should be masked + */ + public boolean isMasked() { + return mask; + } + + /** + * @return the maximum length of this string type + */ + public short getMaxLength() { + return maxLength; + } + + @Override + public String getUrlSafeValueString() { + return UrlSafeUtils.urlEncode(getValue()); + } + + @Override + public void restoreFromUrlSafeValueString(String valueString) { + setValue(UrlSafeUtils.urlDecode(valueString)); + } + + @Override + public MInputType getType() { + return MInputType.STRING; + } + + @Override + protected boolean hasExtraInfo() { + return true; + } + + @Override + protected String getExtraInfoToString() { + return isMasked() + ":" + getMaxLength(); + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (!(other instanceof MStringInput)) { + return false; + } + + MStringInput msi = (MStringInput) other; + return getName().equals(msi.getName()) + && (mask == msi.mask) + && (maxLength == msi.maxLength); + } + + @Override + public int hashCode() { + int result = 23 + 31 * getName().hashCode(); + result = 31 * result + (mask ? 1 : 0); + result = 31 * result + maxLength; + return result; + } +} diff --git a/common/src/main/java/org/apache/sqoop/utils/ClassLoadingUtils.java b/common/src/main/java/org/apache/sqoop/utils/ClassLoadingUtils.java new file mode 100644 index 00000000..a7a870f9 --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/utils/ClassLoadingUtils.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.sqoop.utils; + +import org.apache.log4j.Logger; + +public final class ClassLoadingUtils { + + private static final Logger LOG = Logger.getLogger(ClassLoadingUtils.class); + + public static Class loadClass(String className) { + Class klass = null; + try { + klass = Class.forName(className); + } catch (ClassNotFoundException ex) { + LOG.debug("Exception while loading class: " + className, ex); + } + + if (klass == null) { + // Try the context class loader if one exists + ClassLoader ctxLoader = Thread.currentThread().getContextClassLoader(); + if (ctxLoader != null) { + try { + klass = ctxLoader.loadClass(className); + } catch (ClassNotFoundException ex) { + LOG.debug("Exception while load class: " + className, ex); + } + } + } + + return klass; + } + + private ClassLoadingUtils() { + // Disable explicit object creation + } +} diff --git a/common/src/main/java/org/apache/sqoop/utils/UrlSafeUtils.java b/common/src/main/java/org/apache/sqoop/utils/UrlSafeUtils.java new file mode 100644 index 00000000..f448c94d --- /dev/null +++ b/common/src/main/java/org/apache/sqoop/utils/UrlSafeUtils.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.utils; + +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.net.URLEncoder; + +/** + * Miscellaneous utility methods that help in URL-safe communication over HTTP. + */ +public final class UrlSafeUtils { + + public static final String ENCODING_UTF8 = "UTF-8"; + + public static String urlEncode(String string) { + try { + return URLEncoder.encode(string, ENCODING_UTF8); + } catch (UnsupportedEncodingException uee) { + throw new RuntimeException(uee); + } + } + + public static String urlDecode(String string) { + try { + return URLDecoder.decode(string, ENCODING_UTF8); + } catch (UnsupportedEncodingException uee) { + throw new RuntimeException(uee); + } + } + + + private UrlSafeUtils() { + // Disable explicit object creation + } +} diff --git a/common/src/saveVersion.sh b/common/src/saveVersion.sh new file mode 100755 index 00000000..4310339a --- /dev/null +++ b/common/src/saveVersion.sh @@ -0,0 +1,47 @@ +#!/bin/sh + +# This file is used to generate the annotation of package info that +# records the user, url, revision and timestamp. + +# 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. + +unset LANG +unset LC_CTYPE +version=$1 +outputDirectory=$2 +user=`whoami` +date=`date` +cwd=`pwd` +if [ -d .svn ]; then + revision=`svn info | sed -n -e 's/Last Changed Rev: \(.*\)/\1/p'` + url=`svn info | sed -n -e 's/URL: \(.*\)/\1/p'` +elif [ -d .git ]; then + revision=`git log -1 --pretty=format:"%H"` + hostname=`hostname` + url="git://${hostname}${cwd}" +else + revision="Unknown" + url="file://$cwd" +fi +mkdir -p "$outputDirectory/org/apache/sqoop/common" +cat >"$outputDirectory/org/apache/sqoop/common/package-info.java" < + + + + 4.0.0 + + + org.apache.sqoop + connector + 2.0.0-SNAPSHOT + + + org.apache.sqoop.connector + sqoop-connector-generic-jdbc + Generic JDBC Connector + + + + org.apache.sqoop + sqoop-spi + 2.0.0-SNAPSHOT + + + + diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcConnector.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcConnector.java new file mode 100644 index 00000000..88fc7e54 --- /dev/null +++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcConnector.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.connector.jdbc; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.ResourceBundle; + +import org.apache.sqoop.model.MForm; +import org.apache.sqoop.model.MInput; +import org.apache.sqoop.model.MMapInput; +import org.apache.sqoop.model.MStringInput; +import org.apache.sqoop.connector.spi.SqoopConnector; + +public class GenericJdbcConnector implements SqoopConnector { + + private static final List CONNECTION_FORMS = new ArrayList(); + private static final List JOB_FORMS = new ArrayList(); + + static { + // Build the connection form + List> connFormInputs = new ArrayList>(); + + MStringInput jdbcDriver = new MStringInput( + GenericJdbcConnectorConstants.INPUT_CONN_JDBCDRIVER, false, + (short) 128); + connFormInputs.add(jdbcDriver); + + MStringInput connectString = new MStringInput( + GenericJdbcConnectorConstants.INPUT_CONN_CONNECTSTRING, false, + (short) 128); + connFormInputs.add(connectString); + + MStringInput username = new MStringInput( + GenericJdbcConnectorConstants.INPUT_CONN_USERNAME, false, (short) 36); + connFormInputs.add(username); + + MStringInput password = new MStringInput( + GenericJdbcConnectorConstants.INPUT_CONN_PASSWORD, true, (short) 10); + connFormInputs.add(password); + + MMapInput jdbcProperties = new MMapInput( + GenericJdbcConnectorConstants.INPUT_CONN_JDBCPROPS); + connFormInputs.add(jdbcProperties); + + MForm connForm = new MForm(GenericJdbcConnectorConstants.FORM_CONNECTION, + connFormInputs); + + CONNECTION_FORMS.add(connForm); + } + + @Override + public ResourceBundle getBundle(Locale locale) { + return ResourceBundle.getBundle( + GenericJdbcConnectorConstants.RESOURCE_BUNDLE_NAME, locale); + } + + @Override + public List getConnectionForms() { + return CONNECTION_FORMS; + } + + @Override + public List getJobForms() { + return JOB_FORMS; + } + +} diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcConnectorConstants.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcConnectorConstants.java new file mode 100644 index 00000000..786ff8df --- /dev/null +++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcConnectorConstants.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.connector.jdbc; + +/** + * + */ +public final class GenericJdbcConnectorConstants { + + // Resource bundle name + public static final String RESOURCE_BUNDLE_NAME = + "generic-jdbc-connector-resources"; + + + // Metadata constants + + // Connection form + public static final String FORM_CONNECTION = "form-connection"; + + // Connection form input + public static final String INPUT_CONN_JDBCDRIVER = "inp-conn-jdbcdriver"; + public static final String INPUT_CONN_CONNECTSTRING = + "inp-conn-connectstring"; + public static final String INPUT_CONN_USERNAME = "inp-conn-username"; + public static final String INPUT_CONN_PASSWORD = "inp-conn-password"; + public static final String INPUT_CONN_JDBCPROPS = "inp-conn-jdbc-properties"; + + private GenericJdbcConnectorConstants() { + // Disable explicit object creation + } +} diff --git a/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties b/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties new file mode 100644 index 00000000..10be0e8b --- /dev/null +++ b/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties @@ -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. + +# Generic JDBC Connector Resources + +############################ +# Connection Form +# +form-connection-label = Please provide the following information to create a \ + connection: +form-connection-help = You must supply the information requested in order to \ + create a connection object. + +# jdbc driver +inp-conn-jdbcdriver-label = JDBC Driver Class +inp-conn-jdbcdriver-help = Enter the fully qualified class name of the JDBC \ + driver that will be used for establishing this connection. + +# connect string +inp-conn-connectstring-label = JDBC Connection String +inp-conn-connectstring-help = Enter the value of JDBC connection string to be \ + used by this connector for creating connections. + +# username string +inp-conn-username-label = Username +inp-conn-username-help = Enter the username to be used for connecting to the \ + database. + +# password string +inp-conn-password-label = Password +inp-conn-password-help = Enter the password to be used for connecting to the \ + database. + +# jdbc properties +inp-conn-jdbc-properties-label = JDBC Connection Properties +inp-conn-jdbc-properties-help = Enter any JDBC properties that should be \ + supplied during the creation of connection. + + + diff --git a/connector/connector-generic-jdbc/src/main/resources/sqoopconnector.properties b/connector/connector-generic-jdbc/src/main/resources/sqoopconnector.properties new file mode 100644 index 00000000..a8221f20 --- /dev/null +++ b/connector/connector-generic-jdbc/src/main/resources/sqoopconnector.properties @@ -0,0 +1,18 @@ +# 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. + +# Generic JDBC Connector Properties +org.apache.sqoop.connector.class = org.apache.sqoop.connector.jdbc.GenericJdbcConnector +org.apache.sqoop.connector.name = generic-jdbc-connector diff --git a/connector/connector-mysql-fastpath/pom.xml b/connector/connector-mysql-fastpath/pom.xml new file mode 100644 index 00000000..c854be54 --- /dev/null +++ b/connector/connector-mysql-fastpath/pom.xml @@ -0,0 +1,41 @@ + + + + + 4.0.0 + + + org.apache.sqoop + connector + 2.0.0-SNAPSHOT + + + org.apache.sqoop.connector + sqoop-connector-mysql-fastpath + MySQL Fastpath Connector + + + + org.apache.sqoop + sqoop-spi + 2.0.0-SNAPSHOT + + + diff --git a/connector/connector-mysql-jdbc/pom.xml b/connector/connector-mysql-jdbc/pom.xml new file mode 100644 index 00000000..b8586e1e --- /dev/null +++ b/connector/connector-mysql-jdbc/pom.xml @@ -0,0 +1,46 @@ + + + + + 4.0.0 + + + org.apache.sqoop + connector + 2.0.0-SNAPSHOT + + + org.apache.sqoop.connector + sqoop-connector-mysql-jdbc + MySQL JDBC Connector + + + + org.apache.sqoop + sqoop-spi + 2.0.0-SNAPSHOT + + + org.apache.sqoop.connector + sqoop-connector-generic-jdbc + 2.0.0-SNAPSHOT + + + diff --git a/connector/connector-mysql-jdbc/src/main/java/org/apache/sqoop/connector/mysqljdbc/MySqlJdbcConnector.java b/connector/connector-mysql-jdbc/src/main/java/org/apache/sqoop/connector/mysqljdbc/MySqlJdbcConnector.java new file mode 100644 index 00000000..41e3e42a --- /dev/null +++ b/connector/connector-mysql-jdbc/src/main/java/org/apache/sqoop/connector/mysqljdbc/MySqlJdbcConnector.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.connector.mysqljdbc; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.ResourceBundle; + +import org.apache.sqoop.model.MForm; +import org.apache.sqoop.connector.spi.SqoopConnector; + +public class MySqlJdbcConnector implements SqoopConnector { + + private static final List CONNECTION_FORMS = new ArrayList(); + private static final List JOB_FORMS = new ArrayList(); + + @Override + public ResourceBundle getBundle(Locale locale) { + // TODO Auto-generated method stub + return null; + } + + @Override + public List getConnectionForms() { + return CONNECTION_FORMS; + } + + @Override + public List getJobForms() { + // TODO Auto-generated method stub + return JOB_FORMS; + } +} diff --git a/connector/connector-mysql-jdbc/src/main/resources/sqoopconnector.properties b/connector/connector-mysql-jdbc/src/main/resources/sqoopconnector.properties new file mode 100644 index 00000000..89c4b1cd --- /dev/null +++ b/connector/connector-mysql-jdbc/src/main/resources/sqoopconnector.properties @@ -0,0 +1,18 @@ +# 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. + +# MySQL JDBC COnnector Properties +org.apache.sqoop.connector.class = org.apache.sqoop.connector.mysqljdbc.MySqlJdbcConnector +org.apache.sqoop.connector.name = mysql-jdbc-connector diff --git a/connector/pom.xml b/connector/pom.xml new file mode 100644 index 00000000..d3b3ec8f --- /dev/null +++ b/connector/pom.xml @@ -0,0 +1,41 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + connector + Sqoop Connectors + pom + + + connector-generic-jdbc + connector-mysql-jdbc + connector-mysql-fastpath + + + diff --git a/core/pom.xml b/core/pom.xml new file mode 100644 index 00000000..cbd824f6 --- /dev/null +++ b/core/pom.xml @@ -0,0 +1,72 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-core + Sqoop Core + + + + org.apache.sqoop + sqoop-spi + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-common + 2.0.0-SNAPSHOT + + + commons-dbcp + commons-dbcp + + + junit + junit + test + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/core/src/main/java/org/apache/sqoop/connector/ConnectorError.java b/core/src/main/java/org/apache/sqoop/connector/ConnectorError.java new file mode 100644 index 00000000..2200804b --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/connector/ConnectorError.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.sqoop.connector; + +import org.apache.sqoop.common.ErrorCode; + +public enum ConnectorError implements ErrorCode { + + /** An unknown error has occurred. */ + CONN_0000("An unknown error has occurred"), + + /** The system was not able to initialize the configured connectors. */ + CONN_0001("Unable to initialize connectors"), + + /** No connectors were found in the system. */ + CONN_0002("No connectors were found in the system"), + + /** A problem was encountered while loading the connector configuration. */ + CONN_0003("Failed to load connector configuration"), + + /** A connector configuration file did not include the provider class name.*/ + CONN_0004("Connector configuration did not include provider class name"), + + /** An exception occurred while attempting to instantiate the connector. */ + CONN_0005("Failed to instantiate connector class"), + + /** More than one connectors use the same name resulting in conflict. */ + CONN_0006("More than one connector uses the same name"), + + /** The registration of connector during system initialization failed.*/ + CONN_0007("Connector registration failed"), + + /** The configuration of connector does not specify it's unique name. */ + CONN_0008("No name specified for connector"), + + /** + * A connector is being registered with the same name as what has been + * previously registered. Or the connector being registered is the same but + * it's metadata has changed in an incompatible manner since the last time it + * was registered. + */ + CONN_0009("Attempt to register connector with a name associated with a " + + "previously registered connector; or the connector metadata has " + + "changed since it was registered previously."); + + private final String message; + + private ConnectorError(String message) { + this.message = message; + } + + public String getCode() { + return name(); + } + + public String getMessage() { + return message; + } +} diff --git a/core/src/main/java/org/apache/sqoop/connector/ConnectorHandler.java b/core/src/main/java/org/apache/sqoop/connector/ConnectorHandler.java new file mode 100644 index 00000000..a7f69342 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/connector/ConnectorHandler.java @@ -0,0 +1,120 @@ +/** + * 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.connector; + +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Properties; + +import org.apache.log4j.Logger; +import org.apache.sqoop.core.ConfigurationConstants; +import org.apache.sqoop.model.MConnector; +import org.apache.sqoop.model.MForm; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.connector.spi.SqoopConnector; + +public final class ConnectorHandler { + + private static final Logger LOG = Logger.getLogger(ConnectorHandler.class); + + private final Properties properties = new Properties(); + + private final String connectorUrl; + private final String connectorClassName; + private final String connectorUniqueName; + private final SqoopConnector connector; + + private final MConnector mConnector; + + public ConnectorHandler(URL configFileUrl) { + connectorUrl = configFileUrl.toString(); + try { + properties.load(configFileUrl.openStream()); + } catch (IOException ex) { + throw new SqoopException(ConnectorError.CONN_0003, + configFileUrl.toString(), ex); + } + + LOG.debug("Connector configuration: " + properties); + + connectorClassName = properties.getProperty( + ConfigurationConstants.CONPROP_PROVIDER_CLASS); + + if (connectorClassName == null || connectorClassName.trim().length() == 0) { + throw new SqoopException(ConnectorError.CONN_0004, + ConfigurationConstants.CONPROP_PROVIDER_CLASS); + } + + + connectorUniqueName = properties.getProperty( + ConfigurationConstants.CONNPROP_CONNECTOR_NAME); + + if (connectorUniqueName == null || connectorUniqueName.trim().length() == 0) + { + throw new SqoopException(ConnectorError.CONN_0008, connectorClassName); + } + + Class connectorClass = null; + try { + connectorClass = Class.forName(connectorClassName); + } catch (ClassNotFoundException ex) { + throw new SqoopException(ConnectorError.CONN_0005, + connectorClassName, ex); + } + + try { + connector = (SqoopConnector) connectorClass.newInstance(); + } catch (IllegalAccessException ex) { + throw new SqoopException(ConnectorError.CONN_0005, + connectorClassName, ex); + } catch (InstantiationException ex) { + throw new SqoopException(ConnectorError.CONN_0005, + connectorClassName, ex); + } + + // Initialize Metadata + mConnector = new MConnector(connectorUniqueName, connectorClassName, + connector.getConnectionForms(), connector.getJobForms()); + + if (LOG.isInfoEnabled()) { + LOG.info("Connector [" + connectorClassName + "] initialized."); + } + } + + public String toString() { + return "{" + connectorUniqueName + ":" + connectorClassName + + ":" + connectorUrl + "}"; + } + + public String getUniqueName() { + return connectorUniqueName; + } + + public String getConnectorClassName() { + return connectorClassName; + } + + public String getConnectorUrl() { + return connectorUrl; + } + + public MConnector getMetadata() { + return mConnector; + } +} diff --git a/core/src/main/java/org/apache/sqoop/connector/ConnectorManager.java b/core/src/main/java/org/apache/sqoop/connector/ConnectorManager.java new file mode 100644 index 00000000..585a0dd9 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/connector/ConnectorManager.java @@ -0,0 +1,138 @@ +/** + * 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.connector; + +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.core.ConfigurationConstants; +import org.apache.sqoop.repository.Repository; +import org.apache.sqoop.repository.RepositoryManager; +import org.apache.sqoop.repository.RepositoryTransaction; +import org.apache.sqoop.model.MConnector; + +public class ConnectorManager { + + private static final Logger LOG = Logger.getLogger(ConnectorManager.class); + + private static Map handlerMap = + new HashMap(); + + public static synchronized void initialize() { + if (LOG.isTraceEnabled()) { + LOG.trace("Begin connector manager initialization"); + } + + List connectorConfigs = new ArrayList(); + + try { + Enumeration appPathConfigs = + ConnectorManager.class.getClassLoader().getResources( + ConfigurationConstants.FILENAME_CONNECTOR_PROPERTIES); + + while (appPathConfigs.hasMoreElements()) { + connectorConfigs.add(appPathConfigs.nextElement()); + } + + ClassLoader ctxLoader = Thread.currentThread().getContextClassLoader(); + + if (ctxLoader != null) { + Enumeration ctxPathConfigs = ctxLoader.getResources( + ConfigurationConstants.FILENAME_CONNECTOR_PROPERTIES); + + while (ctxPathConfigs.hasMoreElements()) { + URL configUrl = ctxPathConfigs.nextElement(); + if (!connectorConfigs.contains(configUrl)) { + connectorConfigs.add(configUrl); + } + } + } + + LOG.info("Connector config urls: " + connectorConfigs); + + if (connectorConfigs.size() == 0) { + throw new SqoopException(ConnectorError.CONN_0002); + } + + for (URL url : connectorConfigs) { + ConnectorHandler handler = new ConnectorHandler(url); + ConnectorHandler handlerOld = + handlerMap.put(handler.getUniqueName(), handler); + if (handlerOld != null) { + throw new SqoopException(ConnectorError.CONN_0006, + handler + ", " + handlerOld); + } + } + } catch (IOException ex) { + throw new SqoopException(ConnectorError.CONN_0001, ex); + } + + registerConnectors(); + + if (LOG.isInfoEnabled()) { + LOG.info("Connectors loaded: " + handlerMap); + } + } + + private static synchronized void registerConnectors() { + Repository repository = RepositoryManager.getRepository(); + + RepositoryTransaction rtx = null; + try { + rtx = repository.getTransaction(); + rtx.begin(); + for (String name : handlerMap.keySet()) { + ConnectorHandler handler = handlerMap.get(name); + MConnector connectorMetadata = handler.getMetadata(); + MConnector registeredMetadata = + repository.registerConnector(connectorMetadata); + if (registeredMetadata != null) { + // Verify that the connector metadata is the same + if (!registeredMetadata.equals(connectorMetadata)) { + throw new SqoopException(ConnectorError.CONN_0009, + "To register: " + connectorMetadata + "; already registered: " + + registeredMetadata); + } + } + } + rtx.commit(); + } catch (Exception ex) { + if (rtx != null) { + rtx.rollback(); + } + throw new SqoopException(ConnectorError.CONN_0007, ex); + } finally { + if (rtx != null) { + rtx.close(); + } + } + } + + + public static synchronized void destroy() { + // FIXME + } + +} diff --git a/core/src/main/java/org/apache/sqoop/core/ConfigurationConstants.java b/core/src/main/java/org/apache/sqoop/core/ConfigurationConstants.java new file mode 100644 index 00000000..7fbb15c5 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/ConfigurationConstants.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.core; + +public final class ConfigurationConstants { + + /** + * All configuration keys are prefixed with this: + * org.apache.sqoop. + */ + public static final String PREFIX_GLOBAL_CONFIG = "org.apache.sqoop."; + + /** + * All logging related configuration is prefixed with this: + * org.apache.sqoop.log4j. + */ + public static final String PREFIX_LOG_CONFIG = PREFIX_GLOBAL_CONFIG + + "log4j."; + + /** + * The system property that must be set for specifying the system + * configuration directory: sqoop.config.dir. + */ + public static final String SYSPROP_CONFIG_DIR = "sqoop.config.dir"; + + /** + * Bootstrap configuration property that specifies the system configuration + * provider: sqoop.config.provider. + */ + public static final String BOOTCFG_CONFIG_PROVIDER = "sqoop.config.provider"; + + /** + * Filename for the bootstrap configuration file: + * sqoop_bootstrap.properties. + */ + public static final String FILENAME_BOOTCFG_FILE = + "sqoop_bootstrap.properties"; + + + public static final String FILENAME_CONNECTOR_PROPERTIES = + "sqoopconnector.properties"; + + public static final String CONPROP_PROVIDER_CLASS = + "org.apache.sqoop.connector.class"; + + public static final String CONNPROP_CONNECTOR_NAME = + "org.apache.sqoop.connector.name"; + + + private ConfigurationConstants() { + // Disable explicit object creation + } +} diff --git a/core/src/main/java/org/apache/sqoop/core/ConfigurationListener.java b/core/src/main/java/org/apache/sqoop/core/ConfigurationListener.java new file mode 100644 index 00000000..e1060438 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/ConfigurationListener.java @@ -0,0 +1,24 @@ +/** + * 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.core; + +public interface ConfigurationListener { + + public void configurationChanged(); + +} diff --git a/core/src/main/java/org/apache/sqoop/core/ConfigurationProvider.java b/core/src/main/java/org/apache/sqoop/core/ConfigurationProvider.java new file mode 100644 index 00000000..3dcb853b --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/ConfigurationProvider.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.sqoop.core; + +import java.io.File; +import java.util.Map; +import java.util.Properties; + +public interface ConfigurationProvider { + + public void initialize(File configDir, Properties bootstrapCongiruation); + + public void destroy(); + + public void registerListener(ConfigurationListener listener); + + public Map getConfiguration(); +} diff --git a/core/src/main/java/org/apache/sqoop/core/Context.java b/core/src/main/java/org/apache/sqoop/core/Context.java new file mode 100644 index 00000000..6eeed137 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/Context.java @@ -0,0 +1,67 @@ +/** + * 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.core; + +import java.util.HashMap; +import java.util.Map; + +/** + * Represents a configuration snapshot view for the system. Also provides + * convenience methods for accessing configuration values. + */ +public final class Context { + + private final Map parameters; + + public Context(Map parameters) { + this.parameters = parameters; + } + + public String getString(String key) { + return parameters.get(key); + } + + public String getString(String key, String defaultValue) { + String value = getString(key); + if (value == null || value.trim().length() == 0) { + value = defaultValue; + } + return value; + } + + public boolean getBoolean(String key) { + String value = getString(key); + boolean result = false; + if (value != null) { + result = Boolean.valueOf(value); + } + + return result; + } + + public Map getNestedProperties(String prefix) { + Map subProps = new HashMap(); + for (String key : parameters.keySet()) { + if (key.startsWith(prefix)) { + subProps.put(key.substring(prefix.length()), parameters.get(key)); + } + } + + return subProps; + } +} diff --git a/core/src/main/java/org/apache/sqoop/core/CoreError.java b/core/src/main/java/org/apache/sqoop/core/CoreError.java new file mode 100644 index 00000000..db91d3f2 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/CoreError.java @@ -0,0 +1,67 @@ +/** + * 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.core; + +import org.apache.sqoop.common.ErrorCode; + +public enum CoreError implements ErrorCode { + + /** An unknown error has occurred. */ + CORE_0000("An unknown error has occurred"), + + /** The system was unable to find the configuration directory. */ + CORE_0001("Invalid confiugration directory"), + + /** The system was unable to load bootstrap configuration. */ + CORE_0002("Invalid bootstrap configuration"), + + /** + * The bootstrap configuration did not contain the class name of + * configuration provider. + */ + CORE_0003("No configuration provider set for the system"), + + /** The system was unable locate configuration provider implementation.*/ + CORE_0004("Configuration provider was not found"), + + /** The system was unable to load configuration provider */ + CORE_0005("Unable to load configuration provider"), + + /** + * The PropertiesConfigurationProvider is unable to load the configuration + * properties file. + */ + CORE_0006("Properties configuration provider unable to load config file"), + + /** The configuration system has not been initialized correctly. */ + CORE_0007("System not initialized"); + + private final String message; + + private CoreError(String message) { + this.message = message; + } + + public String getCode() { + return name(); + } + + public String getMessage() { + return message; + } +} diff --git a/core/src/main/java/org/apache/sqoop/core/PropertiesConfigurationProvider.java b/core/src/main/java/org/apache/sqoop/core/PropertiesConfigurationProvider.java new file mode 100644 index 00000000..b66c8098 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/PropertiesConfigurationProvider.java @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.core; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; + +public class PropertiesConfigurationProvider implements ConfigurationProvider { + + private static final Logger LOG = + Logger.getLogger(PropertiesConfigurationProvider.class); + + public static final String CONFIG_FILENAME = "sqoop.properties"; + + private Map configuration = new HashMap(); + + private List listeners = + new ArrayList(); + + private File configFile; + + private ConfigFilePoller poller; + + public PropertiesConfigurationProvider() { + // Default constructor + } + + @Override + public synchronized void registerListener(ConfigurationListener listener) { + listeners.add(listener); + } + + @Override + public synchronized Map getConfiguration() { + Map config = new HashMap(); + config.putAll(configuration); + + return config; + } + + @Override + public synchronized void destroy() { + LOG.info("Shutting down configuration poller thread"); + if (poller != null) { + poller.setShutdown(); + poller.interrupt(); + try { + poller.join(); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + poller = null; + } + + @Override + public synchronized void initialize( + File configDir, Properties bootstrapCongiruation) { + configFile = new File(configDir, CONFIG_FILENAME); + if (!configFile.exists() || !configFile.isFile() || !configFile.canRead()) { + throw new SqoopException(CoreError.CORE_0006, configFile.getPath()); + } + + loadConfiguration(false); // at least one load must succeed + poller = new ConfigFilePoller(configFile); + LOG.info("Starting config file poller thread"); + poller.start(); + } + + private synchronized void loadConfiguration(boolean notifyListeners) { + Properties configProperties = new Properties(); + InputStream fis = null; + try { + fis = new BufferedInputStream(new FileInputStream(configFile)); + configProperties.load(fis); + } catch (Exception ex) { + throw new SqoopException(CoreError.CORE_0006, configFile.getPath()); + } finally { + if (fis != null) { + try { + fis.close(); + } catch (IOException ex) { + LOG.error("Failed to close file stream for configuration", ex); + } + } + } + + configuration.clear(); + Enumeration propNameEnum = configProperties.propertyNames(); + while (propNameEnum.hasMoreElements()) { + String name = (String) propNameEnum.nextElement(); + configuration.put(name, configProperties.getProperty(name)); + } + + if (notifyListeners) { + for (ConfigurationListener configListener : listeners) { + configListener.configurationChanged(); + } + } + } + + private class ConfigFilePoller extends Thread { + private File file; + + private long lastUpdatedAt; + + private boolean shutdown; + + ConfigFilePoller(File configFile) { + this.file = configFile; + lastUpdatedAt = configFile.lastModified(); + this.setName("sqoop-config-file-poller"); + this.setDaemon(true); + } + + synchronized void setShutdown() { + shutdown = true; + } + + private synchronized boolean isShutdown() { + return shutdown; + } + + @Override + public void run() { + + while(true) { + LOG.trace("Checking config file for changes: " + file); + + if (file.lastModified() > lastUpdatedAt) { + LOG.info("Configuration file change detected, attempting to load"); + try { + lastUpdatedAt = file.lastModified(); + loadConfiguration(true); + } catch (Exception ex) { + LOG.error("Exception while loading configuration", ex); + } + } + + try { + Thread.sleep(30); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + + if (isShutdown()) { + break; + } + } + } + } +} diff --git a/core/src/main/java/org/apache/sqoop/core/SqoopConfiguration.java b/core/src/main/java/org/apache/sqoop/core/SqoopConfiguration.java new file mode 100644 index 00000000..3d84d4a3 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/core/SqoopConfiguration.java @@ -0,0 +1,187 @@ +/** + * 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.core; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; +import org.apache.sqoop.common.SqoopException; + +public final class SqoopConfiguration { + + public static final Logger LOG = Logger.getLogger(SqoopConfiguration.class); + + private static File configDir = null; + private static boolean initialized = false; + private static ConfigurationProvider provider = null; + private static Map config = null; + + public synchronized static void initialize() { + if (initialized) { + LOG.warn("Attempt to reinitialize the system, ignoring"); + return; + } + + String configDirPath = System.getProperty( + ConfigurationConstants.SYSPROP_CONFIG_DIR); + if (configDirPath == null || configDirPath.trim().length() == 0) { + throw new SqoopException(CoreError.CORE_0001, "Environment variable " + + ConfigurationConstants.SYSPROP_CONFIG_DIR + " is not set."); + } + + configDir = new File(configDirPath); + if (!configDir.exists() || !configDir.isDirectory()) { + throw new SqoopException(CoreError.CORE_0001, configDirPath); + } + + String bootstrapConfigFilePath = null; + try { + String configDirCanonicalPath = configDir.getCanonicalPath(); + bootstrapConfigFilePath = configDirCanonicalPath + + "/" + ConfigurationConstants.FILENAME_BOOTCFG_FILE; + + } catch (IOException ex) { + throw new SqoopException(CoreError.CORE_0001, configDirPath, ex); + } + + File bootstrapConfig = new File(bootstrapConfigFilePath); + if (!bootstrapConfig.exists() || !bootstrapConfig.isFile() + || !bootstrapConfig.canRead()) { + throw new SqoopException(CoreError.CORE_0002, bootstrapConfigFilePath); + } + + Properties bootstrapProperties = new Properties(); + InputStream bootstrapPropStream = null; + try { + bootstrapPropStream = new FileInputStream(bootstrapConfig); + bootstrapProperties.load(bootstrapPropStream); + } catch (IOException ex) { + throw new SqoopException( + CoreError.CORE_0002, bootstrapConfigFilePath, ex); + } + + String configProviderClassName = bootstrapProperties.getProperty( + ConfigurationConstants.BOOTCFG_CONFIG_PROVIDER); + + if (configProviderClassName == null + || configProviderClassName.trim().length() == 0) { + throw new SqoopException( + CoreError.CORE_0003, ConfigurationConstants.BOOTCFG_CONFIG_PROVIDER); + } + + Class configProviderClass = null; + try { + configProviderClass = Class.forName(configProviderClassName); + } catch (ClassNotFoundException cnfe) { + LOG.warn("Exception while trying to load configuration provider", cnfe); + } + + if (configProviderClass == null) { + ClassLoader ctxLoader = Thread.currentThread().getContextClassLoader(); + if (ctxLoader != null) { + try { + configProviderClass = ctxLoader.loadClass(configProviderClassName); + } catch (ClassNotFoundException cnfe) { + LOG.warn("Exception while trying to load configuration provider: " + + configProviderClassName, cnfe); + } + } + } + + if (configProviderClass == null) { + throw new SqoopException(CoreError.CORE_0004, configProviderClassName); + } + + try { + provider = (ConfigurationProvider) configProviderClass.newInstance(); + } catch (Exception ex) { + throw new SqoopException(CoreError.CORE_0005, + configProviderClassName, ex); + } + + // Initialize the configuration provider + provider.initialize(configDir, bootstrapProperties); + refreshConfiguration(); + provider.registerListener(new CoreConfigurationListener()); + + initialized = true; + } + + public synchronized static Context getContext() { + if (!initialized) { + throw new SqoopException(CoreError.CORE_0007); + } + + Map parameters = new HashMap(); + parameters.putAll(config); + + return new Context(parameters); + } + + public synchronized static void destroy() { + if (provider != null) { + try { + provider.destroy(); + } catch (Exception ex) { + LOG.error("Failed to shutdown configuration provider", ex); + } + } + provider = null; + configDir = null; + config = null; + initialized = false; + } + + private synchronized static void configureLogging() { + Properties props = new Properties(); + for (String key : config.keySet()) { + if (key.startsWith(ConfigurationConstants.PREFIX_LOG_CONFIG)) { + String logConfigKey = key.substring( + ConfigurationConstants.PREFIX_GLOBAL_CONFIG.length()); + props.put(logConfigKey, config.get(key)); + } + } + + PropertyConfigurator.configure(props); + } + + private synchronized static void refreshConfiguration() + { + config = provider.getConfiguration(); + configureLogging(); + } + + private SqoopConfiguration() { + // Disable explicit object creation + } + + public static class CoreConfigurationListener implements ConfigurationListener + { + @Override + public void configurationChanged() { + refreshConfiguration(); + } + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java new file mode 100644 index 00000000..ff9cf7b6 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java @@ -0,0 +1,82 @@ +/** + * 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.repository; + +import java.sql.Connection; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.connector.ConnectorHandler; +import org.apache.sqoop.model.MConnector; + +public class JdbcRepository implements Repository { + + private static final Logger LOG = + Logger.getLogger(JdbcRepository.class); + + private final JdbcRepositoryHandler handler; + private final JdbcRepositoryContext repoContext; + + protected JdbcRepository(JdbcRepositoryHandler handler, + JdbcRepositoryContext repoContext) { + this.handler = handler; + this.repoContext = repoContext; + } + + @Override + public JdbcRepositoryTransaction getTransaction() { + return repoContext.getTransactionFactory().get(); + } + + @Override + public MConnector registerConnector(MConnector mConnector) { + MConnector result = null; + JdbcRepositoryTransaction tx = null; + String connectorUniqueName = mConnector.getUniqueName(); + try { + tx = getTransaction(); + tx.begin(); + Connection conn = tx.getConnection(); + result = handler.findConnector(connectorUniqueName, conn); + if (result == null) { + handler.registerConnector(mConnector, conn); + } else { + if (!result.equals(mConnector)) { + throw new SqoopException(RepositoryError.JDBCREPO_0013, + "given[" + mConnector + "] found[" + result + "]"); + } + } + tx.commit(); + } catch (Exception ex) { + if (tx != null) { + tx.rollback(); + } + if (ex instanceof SqoopException) { + throw (SqoopException) ex; + } + throw new SqoopException(RepositoryError.JDBCREPO_0012, + mConnector.toString(), ex); + } finally { + if (tx != null) { + tx.close(); + } + } + + return result; + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryContext.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryContext.java new file mode 100644 index 00000000..dc01d89d --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryContext.java @@ -0,0 +1,214 @@ +/** + * 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.repository; + +import java.util.Map; +import java.util.Properties; + +import javax.sql.DataSource; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.core.Context; + + +public final class JdbcRepositoryContext { + + private static final Logger LOG = + Logger.getLogger(JdbcRepositoryContext.class); + + private final Context context; + private final String handlerClassName; + private final boolean createSchema; + private final String connectionUrl; + private final String driverClassName; + private final Properties connectionProperties; + private final JdbcTransactionIsolation transactionIsolation; + private final int maxConnections; + + private DataSource dataSource; + private JdbcRepositoryTransactionFactory txFactory; + + public JdbcRepositoryContext(Context context) { + this.context = context; + + handlerClassName = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_HANDLER); + + if (handlerClassName == null || handlerClassName.trim().length() == 0) { + throw new SqoopException(RepositoryError.JDBCREPO_0001, + RepoConfigurationConstants.SYSCFG_REPO_JDBC_HANDLER); + } + + createSchema = context.getBoolean( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_CREATE_SCHEMA); + + connectionUrl = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_URL); + + driverClassName = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_DRIVER); + + String jdbcUserName = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_USER); + + String jdbcPassword = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_PASSWORD); + + connectionProperties = new Properties(); + + Map params = context.getNestedProperties( + RepoConfigurationConstants.PREFIX_SYSCFG_REPO_JDBC_PROPERTIES); + for (String key : params.keySet()) { + connectionProperties.setProperty(key, params.get(key)); + } + + if (jdbcUserName != null) { + Object oldUser = connectionProperties.put("user", jdbcUserName); + if (oldUser != null) { + LOG.warn("Overriding user (" + oldUser + + ") with explicitly specified user (" + jdbcUserName + ")"); + } + } + + if (jdbcPassword != null) { + Object oldPassword = connectionProperties.put("password", jdbcPassword); + if (oldPassword != null) { + LOG.warn("Overriding password from jdbc connection properties with " + + "explicitly specified password."); + } + } + + String txIsolation = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_TX_ISOLATION); + + if (txIsolation == null || txIsolation.trim().length() == 0) { + throw new SqoopException(RepositoryError.JDBCREPO_0004); + } + + try { + transactionIsolation = JdbcTransactionIsolation.getByName(txIsolation); + } catch (IllegalArgumentException ex) { + throw new SqoopException(RepositoryError.JDBCREPO_0004, + txIsolation, ex); + } + + String maxConnStr = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_JDBC_MAX_CONN); + + if (maxConnStr == null || maxConnStr.trim().length() == 0) { + throw new SqoopException(RepositoryError.JDBCREPO_0005, + RepoConfigurationConstants.SYSCFG_REPO_JDBC_MAX_CONN); + } + + int maxConnInt = 0; + try { + maxConnInt = Integer.parseInt(maxConnStr); + } catch (NumberFormatException ex) { + throw new SqoopException(RepositoryError.JDBCREPO_0005, maxConnStr); + } + + if (maxConnInt <= 0) { + throw new SqoopException(RepositoryError.JDBCREPO_0005, maxConnStr); + } + + maxConnections = maxConnInt; + + if (LOG.isInfoEnabled()) { + StringBuilder sb = new StringBuilder("[repo-ctx] "); + sb.append("handler=").append(handlerClassName).append(", "); + sb.append("create-schema=").append(createSchema).append(", "); + sb.append("conn-url=").append(connectionUrl).append(", "); + sb.append("driver=").append(driverClassName).append(", "); + sb.append("user=").append(jdbcUserName).append(", "); + sb.append("password=").append("*****").append(", "); + sb.append("jdbc-props={"); + boolean first = true; + for (String key : params.keySet()) { + if (first) { + first = false; + } else { + sb.append(", "); + } + sb.append(key).append("="); + if (key.equalsIgnoreCase("password")) { + sb.append("*****"); + } else { + sb.append(params.get(key)); + } + } + sb.append("}").append(", "); + sb.append("tx-isolation=").append(transactionIsolation).append(", "); + sb.append("max-conn=").append(maxConnections); + + LOG.info(sb.toString()); + } + } + + void initialize(DataSource source, JdbcRepositoryTransactionFactory factory) { + if (dataSource != null || txFactory != null) { + throw new SqoopException(RepositoryError.JDBCREPO_0011); + } + + dataSource = source; + txFactory = factory; + } + + public DataSource getDataSource() { + return dataSource; + } + + public JdbcRepositoryTransactionFactory getTransactionFactory() { + return txFactory; + } + + public String getHandlerClassName() { + return handlerClassName; + } + + public String getConnectionUrl() { + return connectionUrl; + } + + public String getDriverClass() { + return driverClassName; + } + + public boolean shouldCreateSchema() { + return createSchema; + } + + public JdbcTransactionIsolation getTransactionIsolation() { + return transactionIsolation; + } + + public int getMaximumConnections() { + return maxConnections; + } + + public Properties getConnectionProperties() { + Properties props = new Properties(); + props.putAll(connectionProperties); + + return props; + } + + public Context getContext() { + return context; + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java new file mode 100644 index 00000000..c5d00fe5 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.repository; + +import java.sql.Connection; + +import org.apache.sqoop.model.MConnector; + +public interface JdbcRepositoryHandler { + + public void initialize(JdbcRepositoryContext repoContext); + + public MConnector findConnector(String shortName, Connection conn); + + public void registerConnector(MConnector mc, Connection conn); + + public boolean schemaExists(); + + public void createSchema(); + + public void shutdown(); +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryProvider.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryProvider.java new file mode 100644 index 00000000..021bec06 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryProvider.java @@ -0,0 +1,175 @@ +/** + * 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.repository; + +import java.sql.Driver; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Properties; + +import javax.sql.DataSource; + +import org.apache.commons.dbcp.ConnectionFactory; +import org.apache.commons.dbcp.DriverManagerConnectionFactory; +import org.apache.commons.dbcp.PoolableConnectionFactory; +import org.apache.commons.dbcp.PoolingDataSource; +import org.apache.commons.pool.KeyedObjectPoolFactory; +import org.apache.commons.pool.impl.GenericKeyedObjectPoolFactory; +import org.apache.commons.pool.impl.GenericObjectPool; +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.core.Context; +import org.apache.sqoop.core.SqoopConfiguration; +import org.apache.sqoop.utils.ClassLoadingUtils; + + +public class JdbcRepositoryProvider implements RepositoryProvider { + + private static final Logger LOG = + Logger.getLogger(JdbcRepositoryProvider.class); + + private JdbcRepositoryContext repoContext; + + private Driver driver; + private GenericObjectPool connectionPool; + private KeyedObjectPoolFactory statementPool; + private DataSource dataSource; + + private JdbcRepositoryHandler handler; + private JdbcRepositoryTransactionFactory txFactory; + private JdbcRepository repository; + + + public JdbcRepositoryProvider() { + // Default constructor + } + + @Override + public synchronized void initialize(Context context) { + repoContext = new JdbcRepositoryContext(SqoopConfiguration.getContext()); + + initializeRepositoryHandler(); + + LOG.info("JdbcRepository initialized."); + } + + @Override + public synchronized void destroy() { + try { + connectionPool.close(); + } catch (Exception ex) { + LOG.error("Failed to shutdown connection pool", ex); + } + + handler.shutdown(); + + if (driver != null) { + try { + LOG.info("Deregistering JDBC driver"); + DriverManager.deregisterDriver(driver); + } catch (SQLException ex) { + LOG.error("Failed to deregister driver", ex); + } + } + handler = null; + driver = null; + dataSource = null; + } + + private void initializeRepositoryHandler() { + String jdbcHandlerClassName = repoContext.getHandlerClassName(); + + Class handlerClass = ClassLoadingUtils.loadClass(jdbcHandlerClassName); + + if (handlerClass == null) { + throw new SqoopException(RepositoryError.JDBCREPO_0001, + jdbcHandlerClassName); + } + + try { + handler = (JdbcRepositoryHandler) handlerClass.newInstance(); + } catch (Exception ex) { + throw new SqoopException(RepositoryError.JDBCREPO_0001, + jdbcHandlerClassName, ex); + } + + String connectUrl = repoContext.getConnectionUrl(); + if (connectUrl == null || connectUrl.trim().length() == 0) { + throw new SqoopException(RepositoryError.JDBCREPO_0002); + } + + String jdbcDriverClassName = repoContext.getDriverClass(); + if (jdbcDriverClassName == null || jdbcDriverClassName.trim().length() == 0) + { + throw new SqoopException(RepositoryError.JDBCREPO_0003); + } + + // Initialize a datasource + Class driverClass = ClassLoadingUtils.loadClass(jdbcDriverClassName); + + if (driverClass == null) { + throw new SqoopException(RepositoryError.JDBCREPO_0003, + jdbcDriverClassName); + } + + try { + driver = (Driver) driverClass.newInstance(); + } catch (Exception ex) { + throw new SqoopException(RepositoryError.JDBCREPO_0003, + jdbcDriverClassName, ex); + } + + Properties jdbcProps = repoContext.getConnectionProperties(); + + ConnectionFactory connFactory = + new DriverManagerConnectionFactory(connectUrl, jdbcProps); + + connectionPool = new GenericObjectPool(); + connectionPool.setMaxActive(repoContext.getMaximumConnections()); + + statementPool = new GenericKeyedObjectPoolFactory(null); + + // creating the factor automatically wires the connection pool + new PoolableConnectionFactory(connFactory, connectionPool, statementPool, + /* FIXME validation query */null, false, false, + repoContext.getTransactionIsolation().getCode()); + + dataSource = new PoolingDataSource(connectionPool); + txFactory = new JdbcRepositoryTransactionFactory(dataSource); + + repoContext.initialize(dataSource, txFactory); + + handler.initialize(repoContext); + + if (repoContext.shouldCreateSchema()) { + if (!handler.schemaExists()) { + LOG.info("Creating repository schema objects"); + handler.createSchema(); + } + } + + repository = new JdbcRepository(handler, repoContext); + + LOG.info("JdbcRepositoryProvider initialized"); + } + + @Override + public synchronized Repository getRepository() { + return repository; + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryTransaction.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryTransaction.java new file mode 100644 index 00000000..96c75be4 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryTransaction.java @@ -0,0 +1,158 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.repository; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.SQLWarning; + +import javax.sql.DataSource; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; + +public class JdbcRepositoryTransaction implements RepositoryTransaction { + + private static final Logger LOG = + LogManager.getLogger(JdbcRepositoryTransaction.class); + + private final DataSource dataSource; + private Connection connection; + private JdbcRepositoryTransactionFactory txFactory; + private boolean active = true; + private int count = 0; + + private boolean rollback = false; + + protected JdbcRepositoryTransaction(DataSource dataSource, + JdbcRepositoryTransactionFactory factory) { + this.dataSource = dataSource; + txFactory = factory; + } + + public Connection getConnection() { + return connection; + } + + @Override + public void begin() { + if (!active) { + throw new SqoopException(RepositoryError.JDBCREPO_0006); + } + if (count == 0) { + // Lease a connection now + try { + connection = dataSource.getConnection(); + } catch (SQLException ex) { + throw new SqoopException(RepositoryError.JDBCREPO_0007, ex); + } + // Clear any prior warnings on the connection + try { + connection.clearWarnings(); + } catch (SQLException ex) { + LOG.error("Error while clearing warnings: " + ex.getErrorCode(), ex); + } + } + count++; + LOG.debug("Tx count-begin: " + count + ", rollback: " + rollback); + } + + @Override + public void commit() { + if (!active) { + throw new SqoopException(RepositoryError.JDBCREPO_0006); + } + if (rollback) { + throw new SqoopException(RepositoryError.JDBCREPO_0008); + } + LOG.debug("Tx count-commit: " + count + ", rollback: " + rollback); + } + + @Override + public void rollback() { + if (!active) { + throw new SqoopException(RepositoryError.JDBCREPO_0006); + } + LOG.warn("Marking transaction for rollback"); + rollback = true; + LOG.debug("Tx count-rollback: " + count + ", rollback: " + rollback); + } + + @Override + public void close() { + if (!active) { + throw new SqoopException(RepositoryError.JDBCREPO_0006); + } + count--; + LOG.debug("Tx count-close: " + count + ", rollback: " + rollback); + if (count == 0) { + active = false; + try { + if (rollback) { + LOG.info("Attempting transaction roll-back"); + connection.rollback(); + } else { + LOG.info("Attempting transaction commit"); + connection.commit(); + } + } catch (SQLException ex) { + throw new SqoopException(RepositoryError.JDBCREPO_0009, ex); + } finally { + if (connection != null) { + // Log Warnings + try { + SQLWarning warning = connection.getWarnings(); + if (warning != null) { + StringBuilder sb = new StringBuilder("Connection warnigns: "); + boolean first = true; + while (warning != null) { + if (first) { + first = false; + } else { + sb.append("; "); + } + sb.append("[").append(warning.getErrorCode()).append("] "); + sb.append(warning.getMessage()); + } + LOG.warn(sb.toString()); + } + } catch (SQLException ex) { + LOG.error("Error while retrieving warnigns: " + + ex.getErrorCode(), ex); + } + + // Close Connection + try { + connection.close(); + } catch (SQLException ex) { + LOG.error( + "Unable to close connection: " + ex.getErrorCode(), ex); + } + } + + // Clean up thread local + txFactory.remove(); + + // Destroy local state + connection = null; + txFactory = null; + } + } + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryTransactionFactory.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryTransactionFactory.java new file mode 100644 index 00000000..e4cad4f7 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryTransactionFactory.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.repository; + +import javax.sql.DataSource; + +public class JdbcRepositoryTransactionFactory extends + ThreadLocal { + + private final DataSource dataSource; + + protected JdbcRepositoryTransactionFactory(DataSource dataSource) { + super(); + this.dataSource = dataSource; + } + + @Override + protected JdbcRepositoryTransaction initialValue() { + return new JdbcRepositoryTransaction(dataSource, this); + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcTransactionIsolation.java b/core/src/main/java/org/apache/sqoop/repository/JdbcTransactionIsolation.java new file mode 100644 index 00000000..2b1c8ce2 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/JdbcTransactionIsolation.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.repository; + +import java.sql.Connection; + +public enum JdbcTransactionIsolation { + + READ_UNCOMMITTED("READ_UNCOMMITTED", Connection.TRANSACTION_READ_UNCOMMITTED), + READ_COMMITTED("READ_COMMITTED", Connection.TRANSACTION_READ_COMMITTED), + REPEATABLE_READ("REPEATABLE_READ", Connection.TRANSACTION_REPEATABLE_READ), + SERIALIZABLE("SERIALIZABLE", Connection.TRANSACTION_SERIALIZABLE); + + private final String name; + private final int code; + + private JdbcTransactionIsolation(String name, int code) { + this.name = name; + this.code = code; + } + + public int getCode() { + return code; + } + + public String getName() { + return name; + } + + public String toString() { + return getName(); + } + + public static JdbcTransactionIsolation getByName(String name) { + if (name == null || name.trim().length() == 0) { + return null; + } + return valueOf(name.trim().toUpperCase()); + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/RepoConfigurationConstants.java b/core/src/main/java/org/apache/sqoop/repository/RepoConfigurationConstants.java new file mode 100644 index 00000000..8939f8a1 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/RepoConfigurationConstants.java @@ -0,0 +1,120 @@ +/** + * 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.repository; + +import org.apache.sqoop.core.ConfigurationConstants; + +public final class RepoConfigurationConstants { + + /** + * All repository related configuration is prefixed with this: + * org.apache.sqoop.repository. + */ + public static final String PREFIX_REPO_CONFIG = + ConfigurationConstants.PREFIX_GLOBAL_CONFIG + "repository."; + + /** + * System properties set up by the Repository Manager before initializing + * the provider. + */ + public static final String SYSCFG_REPO_SYSPROP_PREFIX = PREFIX_REPO_CONFIG + + "sysprop."; + + /** + * Class name of the repository implementation specified by: + * org.apache.sqoop.repository.provider + */ + public static final String SYSCFG_REPO_PROVIDER = PREFIX_REPO_CONFIG + + "provider"; + + /** + * Class name for the JDBC repository handler specified by: + * org.apache.sqoop.repository.jdbc.handler. + */ + public static final String SYSCFG_REPO_JDBC_HANDLER = PREFIX_REPO_CONFIG + + "jdbc.handler"; + + /** + * Indicates if the repository should create the schema objects as necessary, + * specified as a boolean value for the key: + * org.apache.sqoop.repository.jdbc.create.schema + */ + public static final String SYSCFG_REPO_JDBC_CREATE_SCHEMA = + PREFIX_REPO_CONFIG + "jdbc.create.schema"; + + /** + * JDBC connection URL specified by: + * org.apache.sqoop.repository.jdbc.url + */ + public static final String SYSCFG_REPO_JDBC_URL = PREFIX_REPO_CONFIG + + "jdbc.url"; + + /** + * JDBC driver to be used, specified by: + * org.apache.sqoop.repository.jdbc.driver + */ + public static final String SYSCFG_REPO_JDBC_DRIVER = PREFIX_REPO_CONFIG + + "jdbc.driver"; + + /** + * JDBC connection user name, specified by: + * org.apache.sqoop.repository.jdbc.user + */ + public static final String SYSCFG_REPO_JDBC_USER = PREFIX_REPO_CONFIG + + "jdbc.user"; + + /** + * JDBC connection password, specified by: + * org.apache.sqoop.repository.jdbc.password + */ + public static final String SYSCFG_REPO_JDBC_PASSWORD = PREFIX_REPO_CONFIG + + "jdbc.password"; + + /** + * JDBC Transaction Isolation, specified by: + * org.apache.sqoop.repository.jdbc.transaction.isolation. The valid + * values include: READ_UNCOMMITTED, READ_COMMITTED, + * REPEATABLE_READ and SERIALIZABLE. + */ + public static final String SYSCFG_REPO_JDBC_TX_ISOLATION = PREFIX_REPO_CONFIG + + "jdbc.transaction.isolation"; + + /** + * JDBC connection pool maximum connections, specified by: + * org.apache.sqoop.repository.jdbc.maximum.connections + */ + public static final String SYSCFG_REPO_JDBC_MAX_CONN = PREFIX_REPO_CONFIG + + "jdbc.maximum.connections"; + + /** + * Prefix that is used to provide any JDBC specific properties for the + * system. Configuration keys which start with this prefix will be stripped + * of the prefix and used as regular properties for JDBC connection + * initialization. The prefix value is + * org.apache.sqoop.repository.jdbc.properties. A property such as + * foo with value bar will be set as: + * org.apache.sqoop.repository.jdbc.properties.foo = bar + */ + public static final String PREFIX_SYSCFG_REPO_JDBC_PROPERTIES = + PREFIX_REPO_CONFIG + "jdbc.properties."; + + + private RepoConfigurationConstants() { + // Disable explicit object creation + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/Repository.java b/core/src/main/java/org/apache/sqoop/repository/Repository.java new file mode 100644 index 00000000..9ad1bca0 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/Repository.java @@ -0,0 +1,43 @@ +/** + * 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.repository; + +import org.apache.sqoop.model.MConnector; + + +/** + * Defines the contract of a Repository used by Sqoop. A Repository allows + * Sqoop to store metadata, statistics and other state relevant to Sqoop + * Jobs in the system. + */ +public interface Repository { + + public RepositoryTransaction getTransaction(); + + /** + * Registers the given connector in the repository. If the connector was + * already registered, its associated metadata is returned from the + * repository. + * + * @param mConnector the connector metadata to be registered + * @return null if the connector was successfully registered or + * a instance of previously registered metadata with the same connector + * unique name. + */ + public MConnector registerConnector(MConnector mConnector); +} diff --git a/core/src/main/java/org/apache/sqoop/repository/RepositoryError.java b/core/src/main/java/org/apache/sqoop/repository/RepositoryError.java new file mode 100644 index 00000000..4f4e1e37 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/RepositoryError.java @@ -0,0 +1,96 @@ +/** + * 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.repository; + +import org.apache.sqoop.common.ErrorCode; + +public enum RepositoryError implements ErrorCode { + + // General Repository Errors: Prefix REPO + + /** An unknown error has occurred. */ + REPO_0000("An unknown error has occurred"), + + /** The system was unable to find or load the repository provider. */ + REPO_0001("Invalid repository provider specified"), + + // JDBC Repository Errors: Prefix JDBCREP + + /** An unknown error has occurred. */ + JDBCREPO_0000("An unknown error has occurred"), + + /** The system was unable to find or load the JDBC repository handler. */ + JDBCREPO_0001("Invalid JDBC Repository Handler specified"), + + /** An invalid JDBC connection URL was specified. */ + JDBCREPO_0002("Invalid JDBC connection URL specified"), + + /** An invalid JDBC driver class name was specified. */ + JDBCREPO_0003("Invalid JDBC driver class specified"), + + /** An invalid JDBC transaction isolation level was specified. */ + JDBCREPO_0004("Invalid JDBC transaction isolation level specified"), + + /** The value specified for maximum connection pool connections is invalid.*/ + JDBCREPO_0005("Invalid maximum connections specified for connection pool"), + + /** The system attempted to use an inactive transaction. */ + JDBCREPO_0006("Transaction is not active"), + + /** + * The system was unable to obtain a connection lease for the + * requested transaction. + */ + JDBCREPO_0007("Unable to lease connection"), + + /** The system attempted to commit a transaction marked for rollback.*/ + JDBCREPO_0008("Attempt to commit a transaction marked for rollback"), + + /** The system was unable to finalize the transaction. */ + JDBCREPO_0009("Failed to finalize transaction"), + + /** The system was not able to deregister the driver during shutdown. */ + JDBCREPO_0010("Unable to deregister driver during shutdown"), + + /** + * An attempt was made to reinitialize already + * initialized JDBC repository context. + */ + JDBCREPO_0011("Attempt to reinitialize JDBC repository context"), + + /** The system was unable to register the connector in its repository. */ + JDBCREPO_0012("Failed to register connector in repository"), + + /** The system found a change in connector metadata that requires upgrade. */ + JDBCREPO_0013("Connector metadata changed - upgrade may be required"); + + + private final String message; + + private RepositoryError(String message) { + this.message = message; + } + + public String getCode() { + return name(); + } + + public String getMessage() { + return message; + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/RepositoryManager.java b/core/src/main/java/org/apache/sqoop/repository/RepositoryManager.java new file mode 100644 index 00000000..17c81a13 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/RepositoryManager.java @@ -0,0 +1,90 @@ +/** + * 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.repository; + +import java.util.Map; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.core.Context; +import org.apache.sqoop.core.SqoopConfiguration; +import org.apache.sqoop.utils.ClassLoadingUtils; + +public final class RepositoryManager { + + private static final Logger LOG = Logger.getLogger(RepositoryManager.class); + + private static RepositoryProvider provider; + + public synchronized static void initialize() { + Context context = SqoopConfiguration.getContext(); + + Map repoSysProps = context.getNestedProperties( + RepoConfigurationConstants.SYSCFG_REPO_SYSPROP_PREFIX); + + LOG.info("Setting system properties: " + repoSysProps); + + for (String key : repoSysProps.keySet()) { + System.setProperty(key, repoSysProps.get(key)); + } + + String repoProviderClassName = context.getString( + RepoConfigurationConstants.SYSCFG_REPO_PROVIDER); + + if (repoProviderClassName == null + || repoProviderClassName.trim().length() == 0) { + throw new SqoopException(RepositoryError.REPO_0001, + RepoConfigurationConstants.SYSCFG_REPO_PROVIDER); + } + + if (LOG.isTraceEnabled()) { + LOG.trace("Repository provider: " + repoProviderClassName); + } + + Class repoProviderClass = + ClassLoadingUtils.loadClass(repoProviderClassName); + + if (repoProviderClass == null) { + throw new SqoopException(RepositoryError.REPO_0001, + repoProviderClassName); + } + + try { + provider = (RepositoryProvider) repoProviderClass.newInstance(); + } catch (Exception ex) { + throw new SqoopException(RepositoryError.REPO_0001, + repoProviderClassName, ex); + } + + provider.initialize(context); + + LOG.info("Repository initialized: OK"); + } + + public static synchronized void destroy() { + try { + provider.destroy(); + } catch (Exception ex) { + LOG.error("Failed to shutdown repository provider", ex); + } + } + + public static synchronized Repository getRepository() { + return provider.getRepository(); + } +} diff --git a/core/src/main/java/org/apache/sqoop/repository/RepositoryProvider.java b/core/src/main/java/org/apache/sqoop/repository/RepositoryProvider.java new file mode 100644 index 00000000..7326536e --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/RepositoryProvider.java @@ -0,0 +1,30 @@ +/** + * 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.repository; + +import org.apache.sqoop.core.Context; + +public interface RepositoryProvider { + + public void initialize(Context context); + + public void destroy(); + + public Repository getRepository(); + +} diff --git a/core/src/main/java/org/apache/sqoop/repository/RepositoryTransaction.java b/core/src/main/java/org/apache/sqoop/repository/RepositoryTransaction.java new file mode 100644 index 00000000..d86d79b5 --- /dev/null +++ b/core/src/main/java/org/apache/sqoop/repository/RepositoryTransaction.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.sqoop.repository; + +/** + * A transaction that can be used to group multiple repository operations into + * a single transaction. + */ +public interface RepositoryTransaction { + + public void begin(); + + public void commit(); + + public void rollback(); + + public void close(); +} diff --git a/core/src/test/java/org/apache/sqoop/core/MockInvalidConfigurationProvider.java b/core/src/test/java/org/apache/sqoop/core/MockInvalidConfigurationProvider.java new file mode 100644 index 00000000..4043e3ee --- /dev/null +++ b/core/src/test/java/org/apache/sqoop/core/MockInvalidConfigurationProvider.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.core; + +import java.io.File; +import java.util.Map; +import java.util.Properties; + +public class MockInvalidConfigurationProvider implements ConfigurationProvider { + + public MockInvalidConfigurationProvider() { + throw new RuntimeException("Cannot instantiate"); + } + + @Override + public void initialize(File configDir, Properties bootstrapCongiruation) { + // TODO Auto-generated method stub + + } + + @Override + public void registerListener(ConfigurationListener listener) { + // TODO Auto-generated method stub + + } + + @Override + public Map getConfiguration() { + // TODO Auto-generated method stub + return null; + } + + @Override + public void destroy() { + // TODO Auto-generated method stub + + } + +} diff --git a/core/src/test/java/org/apache/sqoop/core/TestConfiguration.java b/core/src/test/java/org/apache/sqoop/core/TestConfiguration.java new file mode 100644 index 00000000..bd912eab --- /dev/null +++ b/core/src/test/java/org/apache/sqoop/core/TestConfiguration.java @@ -0,0 +1,165 @@ +/** + * 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.core; + +import java.util.Properties; + +import org.apache.sqoop.common.SqoopException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestConfiguration { + + @Before + public void setUp() throws Exception { + // Unset any configuration dir if it is set by another test + System.getProperties().remove(ConfigurationConstants.SYSPROP_CONFIG_DIR); + SqoopConfiguration.destroy(); + } + + @Test + public void testConfigurationInitFailure() { + boolean success = false; + try { + SqoopConfiguration.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0001); + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testBootstrapConfigurationInitFailure() { + boolean success = false; + try { + String configDirPath = TestUtils.createEmptyConfigDirectory(); + System.setProperty(ConfigurationConstants.SYSPROP_CONFIG_DIR, + configDirPath); + SqoopConfiguration.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0002); + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testConfigurationProviderNotSet() throws Exception { + boolean success = false; + Properties bootProps = new Properties(); + bootProps.setProperty("foo", "bar"); + TestUtils.setupTestConfigurationUsingProperties(bootProps, null); + try { + SqoopConfiguration.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0003); + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testConfigurationProviderInvalid() throws Exception { + boolean success = false; + Properties bootProps = new Properties(); + bootProps.setProperty(ConfigurationConstants.BOOTCFG_CONFIG_PROVIDER, + "foobar"); + TestUtils.setupTestConfigurationUsingProperties(bootProps, null); + try { + SqoopConfiguration.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0004); + + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testConfiugrationProviderCannotLoad() throws Exception { + boolean success = false; + Properties bootProps = new Properties(); + bootProps.setProperty(ConfigurationConstants.BOOTCFG_CONFIG_PROVIDER, + MockInvalidConfigurationProvider.class.getCanonicalName()); + TestUtils.setupTestConfigurationUsingProperties(bootProps, null); + try { + SqoopConfiguration.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0005); + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testPropertiesConfigProviderNoFile() throws Exception { + boolean success = false; + Properties bootProps = new Properties(); + bootProps.setProperty(ConfigurationConstants.BOOTCFG_CONFIG_PROVIDER, + PropertiesConfigurationProvider.class.getCanonicalName()); + TestUtils.setupTestConfigurationUsingProperties(bootProps, null); + try { + SqoopConfiguration.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0006); + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testSystemNotInitialized() throws Exception { + boolean success = false; + try { + SqoopConfiguration.getContext(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + CoreError.CORE_0007); + success = true; + } + + Assert.assertTrue(success); + } + + @Test + public void testConfigurationInitSuccess() throws Exception { + TestUtils.setupTestConfigurationWithExtraConfig(null, null); + SqoopConfiguration.initialize(); + } +} diff --git a/core/src/test/java/org/apache/sqoop/core/TestUtils.java b/core/src/test/java/org/apache/sqoop/core/TestUtils.java new file mode 100644 index 00000000..5b230bae --- /dev/null +++ b/core/src/test/java/org/apache/sqoop/core/TestUtils.java @@ -0,0 +1,183 @@ +/** + * 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.core; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.Enumeration; +import java.util.Properties; + +import org.apache.log4j.Logger; + +public class TestUtils { + + private static final Logger LOG = Logger.getLogger(TestUtils.class); + + public static final String NEWLINE = + System.getProperty("line.separator", "\n"); + + public static String createEmptyConfigDirectory() throws Exception { + File tempDir = null; + File targetDir = new File("target"); + if (targetDir.exists() && targetDir.isDirectory()) { + tempDir = targetDir; + } else { + tempDir = new File(System.getProperty("java.io.tmpdir")); + } + + File tempFile = File.createTempFile("test", "config", tempDir); + String tempConfigDirPath = tempFile.getCanonicalPath() + ".dir/config"; + if (!tempFile.delete()) { + throw new Exception("Unable to delete tempfile: " + tempFile); + } + + File tempConfigDir = new File(tempConfigDirPath); + if (!tempConfigDir.mkdirs()) { + throw new Exception("Unable to create temp config dir: " + + tempConfigDirPath); + } + + return tempConfigDirPath; + } + + public static void setupTestConfigurationUsingProperties( + Properties bootstrapProps, Properties props) + throws Exception { + + String tempConfigDirPath = createEmptyConfigDirectory(); + File tempConfigDir = new File(tempConfigDirPath); + + File bootconfigFile = new File(tempConfigDir, + ConfigurationConstants.FILENAME_BOOTCFG_FILE); + + if (!bootconfigFile.createNewFile()) { + throw new Exception("Unable to create config file: " + bootconfigFile); + } + + if (bootstrapProps != null) { + BufferedWriter bootconfigWriter = null; + try { + bootconfigWriter = new BufferedWriter(new FileWriter(bootconfigFile)); + + Enumeration bootstrapPropNames = bootstrapProps.propertyNames(); + while (bootstrapPropNames.hasMoreElements()) { + String name = (String) bootstrapPropNames.nextElement(); + String value = bootstrapProps.getProperty(name); + bootconfigWriter.write(name + " = " + value + NEWLINE); + } + + bootconfigWriter.flush(); + } finally { + if (bootconfigWriter != null) { + try { + bootconfigWriter.close(); + } catch (IOException ex) { + LOG.error("Failed to close config file writer", ex); + } + } + } + } + + File sysconfigFile = new File(tempConfigDir, + PropertiesConfigurationProvider.CONFIG_FILENAME); + + if (props != null) { + BufferedWriter sysconfigWriter = null; + try { + sysconfigWriter = new BufferedWriter(new FileWriter(sysconfigFile)); + + Enumeration propNameEnum = props.propertyNames(); + while (propNameEnum.hasMoreElements()) { + String name = (String) propNameEnum.nextElement(); + String value = props.getProperty(name); + sysconfigWriter.write(name + " = " + value + NEWLINE); + } + sysconfigWriter.flush(); + } finally { + if (sysconfigWriter != null) { + try { + sysconfigWriter.close(); + } catch (IOException ex) { + LOG.error("Failed to close log config file writer", ex); + } + } + } + } + System.setProperty(ConfigurationConstants.SYSPROP_CONFIG_DIR, + tempConfigDirPath); + } + + /** + * Sets up the test configuration using any properties specified in the + * test file test_config.properties. If the parameter extraConfig + * is specified, it is added to these properties. Consequently any property + * that exists in both the test_config.properties and the supplied extra + * properties will retain the value from the later. + * + * @param extraConfig any properties that you would like to set in the system + * @throws Exception + */ + public static void setupTestConfigurationWithExtraConfig( + Properties extraBootstrapConfig, Properties extraConfig) throws Exception + { + + Properties props = new Properties(); + + InputStream istream = null; + try { + istream = ClassLoader.getSystemResourceAsStream("test_config.properties"); + props.load(istream); + } finally { + if (istream != null) { + try { + istream.close(); + } catch (Exception ex) { + LOG.warn("Failed to close input stream", ex); + } + } + } + + if (props.size() == 0) { + throw new Exception("Unable to load test_config.properties"); + } + + if (extraConfig != null && extraConfig.size() > 0) { + props.putAll(extraConfig); + } + + Properties bootstrapProps = new Properties(); + bootstrapProps.setProperty("sqoop.config.provider", + PropertiesConfigurationProvider.class.getCanonicalName()); + + if (extraBootstrapConfig != null && extraBootstrapConfig.size() > 0) { + bootstrapProps.putAll(extraBootstrapConfig); + } + + setupTestConfigurationUsingProperties(bootstrapProps, props); + } + + + private TestUtils() { + // Disable explicit object creation + } +} diff --git a/core/src/test/java/org/apache/sqoop/repository/TestRepositoryManager.java b/core/src/test/java/org/apache/sqoop/repository/TestRepositoryManager.java new file mode 100644 index 00000000..a95c2855 --- /dev/null +++ b/core/src/test/java/org/apache/sqoop/repository/TestRepositoryManager.java @@ -0,0 +1,50 @@ +/** + * 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.repository; + +import java.util.Properties; + +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.core.ConfigurationConstants; +import org.apache.sqoop.core.PropertiesConfigurationProvider; +import org.apache.sqoop.core.SqoopConfiguration; +import org.apache.sqoop.core.TestUtils; +import org.junit.Assert; +import org.junit.Test; + +public class TestRepositoryManager { + + @Test + public void testSystemNotInitialized() throws Exception { + // Unset any configuration dir if it is set by another test + System.getProperties().remove(ConfigurationConstants.SYSPROP_CONFIG_DIR); + Properties bootProps = new Properties(); + bootProps.setProperty(ConfigurationConstants.BOOTCFG_CONFIG_PROVIDER, + PropertiesConfigurationProvider.class.getCanonicalName()); + Properties configProps = new Properties(); + TestUtils.setupTestConfigurationUsingProperties(bootProps, configProps); + try { + SqoopConfiguration.initialize(); + RepositoryManager.initialize(); + } catch (Exception ex) { + Assert.assertTrue(ex instanceof SqoopException); + Assert.assertSame(((SqoopException) ex).getErrorCode(), + RepositoryError.REPO_0001); + } + } +} diff --git a/core/src/test/resources/test_config.properties b/core/src/test/resources/test_config.properties new file mode 100644 index 00000000..4ad12679 --- /dev/null +++ b/core/src/test/resources/test_config.properties @@ -0,0 +1,25 @@ +# 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. + +# General configuration +org.apache.sqoop.foo = bar + + +# Logging Configuration +org.apache.sqoop.log4j.rootLogger=warn, console +org.apache.sqoop.log4j.logger.org.apache.sqoop=debug,console +org.apache.sqoop.log4j.appender.console=org.apache.log4j.ConsoleAppender +org.apache.sqoop.log4j.appender.console.layout=org.apache.log4j.PatternLayout +org.apache.sqoop.log4j.appender.console.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/dist/pom.xml b/dist/pom.xml new file mode 100644 index 00000000..0db8e72c --- /dev/null +++ b/dist/pom.xml @@ -0,0 +1,33 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-dist + Sqoop Distribution + diff --git a/docs/pom.xml b/docs/pom.xml new file mode 100644 index 00000000..8f2b4cf6 --- /dev/null +++ b/docs/pom.xml @@ -0,0 +1,47 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-docs + Sqoop Documentation + war + + + + org.apache.sqoop + sqoop-core + 2.0.0-SNAPSHOT + + + + + sqoopdocs + + + diff --git a/docs/src/main/webapp/WEB-INF/web.xml b/docs/src/main/webapp/WEB-INF/web.xml new file mode 100644 index 00000000..aa698777 --- /dev/null +++ b/docs/src/main/webapp/WEB-INF/web.xml @@ -0,0 +1,25 @@ + + + + + Apache Sqoop + + + diff --git a/docs/src/main/webapp/index.html b/docs/src/main/webapp/index.html new file mode 100644 index 00000000..06513dd4 --- /dev/null +++ b/docs/src/main/webapp/index.html @@ -0,0 +1,23 @@ + + + +Apache Sqoop + +Apache Sqoop + + diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000..2a61e68d --- /dev/null +++ b/pom.xml @@ -0,0 +1,309 @@ + + + + 4.0.0 + + org.apache + apache + 9 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + Apache Sqoop + Sqoop + pom + + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + Apache Software Foundation + http://www.apache.org/ + + + + JIRA + https://issues.apache.org/jira/browse/sqoop + + + + Jenkins + https://builds.apache.org/job/sqoop/ + + + 2011 + + + + sqoop-user + incubator-sqoop-user-subscribe@apache.org + incubator-sqoop-user-unsubscribe@apache.org + sqoop-user@incubator.apache.org + http://mail-archives.apache.org/mod_mbox/incubator-sqoop-user/ + + + sqoop-dev + incubator-sqoop-dev-subscribe@apache.org + incubator-sqoop-dev-unsubscribe@apache.org + sqoop-dev@incubator.apache.org + http://mail-archives.apache.org/mod_mbox/incubator-sqoop-dev/ + + + sqoop-commits + incubator-sqoop-commits-subscribe@apache.org + incubator-sqoop-commits-unsubscribe@apache.org + sqoop-commits@incubator.apache.org + http://mail-archives.apache.org/mod_mbox/incubator-sqoop-commits/ + + + + + scm:svn:http://svn.apache.org/repos/asf/incubator/sqoop/branches/sqoop2 + scm:svn:https://svn.apache.org/repos/asf/incubator/sqoop/branches/sqoop2/ + http://svn.apache.org/viewvc/incubator/sqoop/branches/sqoop2/ + + + + UTF-8 + 1.6 + 1.6 + 1.2.16 + 1.1 + 1.4 + 10.8.2.2 + 4.9 + + + + + log4j + log4j + + + + + + + log4j + log4j + ${log4j.version} + + + javax.servlet + servlet-api + 2.5 + + + com.googlecode.json-simple + json-simple + ${json-simple.version} + + + commons-dbcp + commons-dbcp + ${commons-dbcp.version} + + + org.apache.derby + derby + ${derby.version} + + + junit + junit + ${junit.version} + + + + + + common + spi + core + repository + server + client + docs + connector + dist + + + + + + org.apache.rat + apache-rat-plugin + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + ${maven.compile.source} + ${maven.compile.target} + + + + org.apache.rat + apache-rat-plugin + 0.8 + + + header-check + verify + + check + + + + .git/ + .gitignore + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.3.2 + + + + + + + + + abayer + Andrew Bayer + Cloudera + + committer + PPMC member + + + + ahmed + Ahmed Radwan + Cloudera + + committer + PPMC member + + + + arvind + Arvind Prabhakar + Cloudera + + committer + PPMC member + + + + blee + Bilung Lee + Cloudera + + committer + + + + gcottman + Greg Cottman + Quest + + committer + PPMC member + + + + guylemar + Guy le Mar + Quest + + committer + PPMC member + + + + jarcec + Jaroslav Cecho + AVG Technologies + + committer + + + + jmhsieh + Jonathan Hsieh + Cloudera + + committer + PPMC member + + + + kimballa + Aaron Kimball + Odiago + + committer + PPMC member + + + + pzimdars + Paul Zimdars + JPL + + committer + PPMC member + + + + rvs + Roman Shaposhnik + Cloudera + + committer + PPMC member + + + + posix4e + OPower + + committer + + + + diff --git a/repository/pom.xml b/repository/pom.xml new file mode 100644 index 00000000..e3345c4b --- /dev/null +++ b/repository/pom.xml @@ -0,0 +1,39 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + repository + Sqoop Repository + pom + + + repository-derby + + + diff --git a/repository/repository-derby/pom.xml b/repository/repository-derby/pom.xml new file mode 100644 index 00000000..0415835b --- /dev/null +++ b/repository/repository-derby/pom.xml @@ -0,0 +1,62 @@ + + + + + 4.0.0 + + + org.apache.sqoop + repository + 2.0.0-SNAPSHOT + + + org.apache.sqoop.repository + sqoop-repository-derby + Sqoop Derby Repository + + + + org.apache.sqoop + sqoop-core + 2.0.0-SNAPSHOT + + + + org.apache.derby + derby + + + + junit + junit + test + + + + org.apache.sqoop + sqoop-core + 2.0.0-SNAPSHOT + test-jar + test + + + + + diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepoConfigurationConstants.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepoConfigurationConstants.java new file mode 100644 index 00000000..beb983c8 --- /dev/null +++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepoConfigurationConstants.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.repository.derby; + +public final class DerbyRepoConfigurationConstants { + + public static final String PREFIX_DERBY = "derby."; + + private DerbyRepoConfigurationConstants() { + // Disable explicit object creation + } +} diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepoError.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepoError.java new file mode 100644 index 00000000..7bd5f6f2 --- /dev/null +++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepoError.java @@ -0,0 +1,122 @@ +/** + * 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.repository.derby; + +import org.apache.sqoop.common.ErrorCode; + +public enum DerbyRepoError implements ErrorCode { + + /** An unknown error has occurred. */ + DERBYREPO_0000("An unknown error has occurred"), + + /** The Derby Repository handler was unable to determine if schema exists.*/ + DERBYREPO_0001("Unable to determine if schema exists"), + + /** The system was unable to shutdown embedded derby repository server. */ + DERBYREPO_0002("Unable to shutdown embedded Derby instance"), + + /** The system was unable to run the specified query. */ + DERBYREPO_0003("Unable to run specified query"), + + /** The system was unable to query the repository for connector metadata. */ + DERBYREPO_0004("Unable to retrieve connector metadata"), + + /** The metadata repository contains more than one connector with same name */ + DERBYREPO_0005("Invalid metadata state - multiple connectors with name"), + + /** The system does not support the given input type.*/ + DERBYREPO_0006("Unknown input type encountered"), + + /** The system does not support the given form type.*/ + DERBYREPO_0007("Unknown form type encountered"), + + /** No input metadata was found for the given form. */ + DERBYREPO_0008("The form contains no input metadata"), + + /** The system could not load the form due to unexpected position of input.*/ + DERBYREPO_0009("The form input retrieved does not match expected position"), + + /** + * The system could not load the connector due to unexpected position + * of form. + */ + DERBYREPO_0010("The form retrieved does not match expteced position"), + + /** + * The system was not able to register connector metadata due to a + * pre-assigned persistence identifier. + */ + DERBYREPO_0011("Connector metadata cannot have preassigned persistence id"), + + /** + * The system was unable to register connector metadata due to an unexpected + * update count. + */ + DERBYREPO_0012("Unexpected update count on connector registration"), + + /** + * The system was unable to register connector metadata due to a failure to + * retrieve the generated identifier. + */ + DERBYREPO_0013("Unable to retrieve generated identifier for new connector"), + + /** + * The system was unable to register connector metadata due to a server + * error. + */ + DERBYREPO_0014("Registration of connector metadata failed"), + + /** + * The system was not able to register connector metadata due to an unexpected + * update count. + */ + DERBYREPO_0015("Unexpected update count on form registration"), + + /** + * The system was unable to register connector metadata due to a failure to + * retrieve the generated identifier for a form. + */ + DERBYREPO_0016("Unable to retrieve generated identifier for form"), + + /** + * The system was unable to register connector metadata due to an unexpected + * update count for form input registration. + */ + DERBYREPO_0017("Unexpected update count for form input"), + + /** + * The system was unable to register connector metadata due to a failure to + * retrieve the generated identifier for a form input. + */ + DERBYREPO_0018("Unable to retrieve generated identifier for form input"); + + + private final String message; + + private DerbyRepoError(String message) { + this.message = message; + } + + public String getCode() { + return name(); + } + + public String getMessage() { + return message; + } +} diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java new file mode 100644 index 00000000..01320b62 --- /dev/null +++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java @@ -0,0 +1,488 @@ +/** + * 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.repository.derby; + +import static org.apache.sqoop.repository.derby.DerbySchemaQuery.*; +import static org.apache.sqoop.repository.derby.DerbySchemaQuery.QUERY_CREATE_TABLE_SQ_CONNECTOR; +import static org.apache.sqoop.repository.derby.DerbySchemaQuery.QUERY_CREATE_TABLE_SQ_FORM; +import static org.apache.sqoop.repository.derby.DerbySchemaQuery.QUERY_CREATE_TABLE_SQ_INPUT; +import static org.apache.sqoop.repository.derby.DerbySchemaQuery.STMT_FETCH_BASE_CONNECTOR; +import static org.apache.sqoop.repository.derby.DerbySchemaQuery.STMT_FETCH_FORM; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; + +import javax.sql.DataSource; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.model.MConnector; +import org.apache.sqoop.model.MForm; +import org.apache.sqoop.model.MFormType; +import org.apache.sqoop.model.MInput; +import org.apache.sqoop.model.MInputType; +import org.apache.sqoop.model.MMapInput; +import org.apache.sqoop.model.MStringInput; +import org.apache.sqoop.repository.JdbcRepositoryContext; +import org.apache.sqoop.repository.JdbcRepositoryHandler; +import org.apache.sqoop.repository.JdbcRepositoryTransactionFactory; + +public class DerbyRepositoryHandler implements JdbcRepositoryHandler { + + private static final Logger LOG = + Logger.getLogger(DerbyRepositoryHandler.class); + + private static final String SCHEMA_SQOOP = "SQOOP"; + + private static final String QUERY_SYSSCHEMA_SQOOP = + "SELECT SCHEMAID FROM SYS.SYSSCHEMAS WHERE SCHEMANAME = '" + + SCHEMA_SQOOP + "'"; + + private static final String EMBEDDED_DERBY_DRIVER_CLASSNAME = + "org.apache.derby.jdbc.EmbeddedDriver"; + + + private JdbcRepositoryContext repoContext; + private DataSource dataSource; + private JdbcRepositoryTransactionFactory txFactory; + + @Override + public void registerConnector(MConnector mc, Connection conn) { + if (mc.getPersistenceId() != -1L) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0011, + mc.getUniqueName()); + } + + PreparedStatement baseConnectorStmt = null; + PreparedStatement baseFormStmt = null; + PreparedStatement baseInputStmt = null; + try { + baseConnectorStmt = conn.prepareStatement(STMT_INSERT_CONNECTOR_BASE, + Statement.RETURN_GENERATED_KEYS); + baseConnectorStmt.setString(1, mc.getUniqueName()); + baseConnectorStmt.setString(2, mc.getClassName()); + + int baseConnectorCount = baseConnectorStmt.executeUpdate(); + if (baseConnectorCount != 1) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0012, + new Integer(baseConnectorCount).toString()); + } + + ResultSet rsetConnectorId = baseConnectorStmt.getGeneratedKeys(); + + if (!rsetConnectorId.next()) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0013); + } + + long connectorId = rsetConnectorId.getLong(1); + mc.setPersistenceId(connectorId); + + baseFormStmt = conn.prepareStatement(STMT_INSERT_FORM_BASE, + Statement.RETURN_GENERATED_KEYS); + + baseInputStmt = conn.prepareStatement(STMT_INSERT_INPUT_BASE, + Statement.RETURN_GENERATED_KEYS); + + // Insert connection forms + registerForms(connectorId, mc.getConnectionForms(), + MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt); + + registerForms(connectorId, mc.getJobForms(), + MFormType.JOB.name(), baseFormStmt, baseInputStmt); + + } catch (SQLException ex) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0014, + mc.toString(), ex); + } finally { + if (baseConnectorStmt != null) { + try { + baseConnectorStmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close base connector statement", ex); + } + } + if (baseFormStmt != null) { + try { + baseFormStmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close base form statement", ex); + } + } + if (baseInputStmt != null) { + try { + baseInputStmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close base input statement", ex); + } + } + } + } + + private void registerForms(long connectorId, List forms, String type, + PreparedStatement baseFormStmt, PreparedStatement baseInputStmt) + throws SQLException { + short formIndex = 0; + for (MForm form : forms) { + baseFormStmt.setLong(1, connectorId); + baseFormStmt.setString(2, form.getName()); + baseFormStmt.setString(3, type); + baseFormStmt.setShort(4, formIndex++); + + int baseFormCount = baseFormStmt.executeUpdate(); + if (baseFormCount != 1) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0015, + new Integer(baseFormCount).toString()); + } + ResultSet rsetFormId = baseFormStmt.getGeneratedKeys(); + if (!rsetFormId.next()) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0016); + } + + long formId = rsetFormId.getLong(1); + form.setPersistenceId(formId); + + // Insert all the inputs + List> inputs = form.getInputs(); + registerFormInputs(formId, inputs, baseInputStmt); + } + } + + private void registerFormInputs(long formId, List> inputs, + PreparedStatement baseInputStmt) throws SQLException { + short inputIndex = 0; + for (MInput input : inputs) { + baseInputStmt.setString(1, input.getName()); + baseInputStmt.setLong(2, formId); + baseInputStmt.setShort(3, inputIndex++); + baseInputStmt.setString(4, input.getType().name()); + if (input.getType().equals(MInputType.STRING)) { + MStringInput strInput = (MStringInput) input; + baseInputStmt.setBoolean(5, strInput.isMasked()); + baseInputStmt.setShort(6, strInput.getMaxLength()); + } + int baseInputCount = baseInputStmt.executeUpdate(); + if (baseInputCount != 1) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0017, + new Integer(baseInputCount).toString()); + } + + ResultSet rsetInputId = baseInputStmt.getGeneratedKeys(); + if (!rsetInputId.next()) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0018); + } + + long inputId = rsetInputId.getLong(1); + input.setPersistenceId(inputId); + } + } + + @Override + public synchronized void initialize(JdbcRepositoryContext ctx) { + repoContext = ctx; + dataSource = repoContext.getDataSource(); + txFactory = repoContext.getTransactionFactory(); + LOG.info("DerbyRepositoryHandler initialized."); + } + + @Override + public synchronized void shutdown() { + String driver = repoContext.getDriverClass(); + if (driver != null && driver.equals(EMBEDDED_DERBY_DRIVER_CLASSNAME)) { + // Using embedded derby. Needs explicit shutdown + String connectUrl = repoContext.getConnectionUrl(); + if (connectUrl.startsWith("jdbc:derby:")) { + int index = connectUrl.indexOf(";"); + String baseUrl = null; + if (index != -1) { + baseUrl = connectUrl.substring(0, index+1); + } else { + baseUrl = connectUrl + ";"; + } + String shutDownUrl = baseUrl + "shutdown=true"; + + LOG.debug("Attempting to shutdown embedded Derby using URL: " + + shutDownUrl); + + try { + DriverManager.getConnection(shutDownUrl); + } catch (SQLException ex) { + // Shutdown for one db instance is expected to raise SQL STATE 45000 + if (ex.getErrorCode() != 45000) { + throw new SqoopException( + DerbyRepoError.DERBYREPO_0002, shutDownUrl, ex); + } + LOG.info("Embedded Derby shutdown raised SQL STATE " + + "45000 as expected."); + } + } else { + LOG.warn("Even though embedded Derby drvier was loaded, the connect " + + "URL is of an unexpected form: " + connectUrl + ". Therfore no " + + "attempt will be made to shutdown embedded Derby instance."); + } + + } + } + + public void createSchema() { + runQuery(QUERY_CREATE_SCHEMA_SQOOP); + runQuery(QUERY_CREATE_TABLE_SQ_CONNECTOR); + runQuery(QUERY_CREATE_TABLE_SQ_FORM); + runQuery(QUERY_CREATE_TABLE_SQ_INPUT); + } + + public boolean schemaExists() { + Connection connection = null; + Statement stmt = null; + try { + connection = dataSource.getConnection(); + stmt = connection.createStatement(); + ResultSet rset = stmt.executeQuery(QUERY_SYSSCHEMA_SQOOP); + + if (!rset.next()) { + LOG.warn("Schema for SQOOP does not exist"); + return false; + } + String sqoopSchemaId = rset.getString(1); + LOG.debug("SQOOP schema ID: " + sqoopSchemaId); + + connection.commit(); + } catch (SQLException ex) { + if (connection != null) { + try { + connection.rollback(); + } catch (SQLException ex2) { + LOG.error("Unable to rollback transaction", ex2); + } + } + throw new SqoopException(DerbyRepoError.DERBYREPO_0001, ex); + } finally { + if (stmt != null) { + try { + stmt.close(); + } catch(SQLException ex) { + LOG.error("Unable to close schema lookup stmt", ex); + } + } + if (connection != null) { + try { + connection.close(); + } catch (SQLException ex) { + LOG.error("Unable to close connection", ex); + } + } + } + + return true; + } + + private void runQuery(String query) { + Connection connection = null; + Statement stmt = null; + try { + connection = dataSource.getConnection(); + stmt = connection.createStatement(); + if (stmt.execute(query)) { + ResultSet rset = stmt.getResultSet(); + int count = 0; + while (rset.next()) { + count++; + } + LOG.info("QUERY(" + query + ") produced unused resultset with " + + count + " rows"); + } else { + int updateCount = stmt.getUpdateCount(); + LOG.info("QUERY(" + query + ") Update count: " + updateCount); + } + connection.commit(); + } catch (SQLException ex) { + try { + connection.rollback(); + } catch (SQLException ex2) { + LOG.error("Unable to rollback transaction", ex2); + } + throw new SqoopException(DerbyRepoError.DERBYREPO_0003, + query, ex); + } finally { + if (stmt != null) { + try { + stmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close statement", ex); + } + if (connection != null) { + try { + connection.close(); + } catch (SQLException ex) { + LOG.error("Unable to close connection", ex); + } + } + } + } + } + + @Override + public MConnector findConnector(String shortName, Connection conn) { + if (LOG.isDebugEnabled()) { + LOG.debug("Looking up connector: " + shortName); + } + MConnector mc = null; + PreparedStatement baseConnectorFetchStmt = null; + PreparedStatement formFetchStmt = null; + PreparedStatement inputFetchStmt = null; + try { + baseConnectorFetchStmt = conn.prepareStatement(STMT_FETCH_BASE_CONNECTOR); + baseConnectorFetchStmt.setString(1, shortName); + ResultSet rsetBaseConnector = baseConnectorFetchStmt.executeQuery(); + + if (!rsetBaseConnector.next()) { + LOG.debug("No connector found by name: " + shortName); + return null; + } + + long connectorId = rsetBaseConnector.getLong(1); + String connectorName = rsetBaseConnector.getString(2); + String connectorClassName = rsetBaseConnector.getString(3); + + List connectionForms = new ArrayList(); + List jobForms = new ArrayList(); + + mc = new MConnector(connectorName, connectorClassName, + connectionForms, jobForms); + mc.setPersistenceId(connectorId); + + if (rsetBaseConnector.next()) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0005, shortName); + } + + formFetchStmt = conn.prepareStatement(STMT_FETCH_FORM); + formFetchStmt.setLong(1, connectorId); + + inputFetchStmt = conn.prepareStatement(STMT_FETCH_INPUT); + + ResultSet rsetForm = formFetchStmt.executeQuery(); + while (rsetForm.next()) { + long formId = rsetForm.getLong(1); + long formConnectorId = rsetForm.getLong(2); + String formName = rsetForm.getString(3); + String formType = rsetForm.getString(4); + int formIndex = rsetForm.getInt(5); + List> formInputs = new ArrayList>(); + + MForm mf = new MForm(formName, formInputs); + mf.setPersistenceId(formId); + + inputFetchStmt.setLong(1, formId); + + ResultSet rsetInput = inputFetchStmt.executeQuery(); + while (rsetInput.next()) { + long inputId = rsetInput.getLong(1); + String inputName = rsetInput.getString(2); + long inputForm = rsetInput.getLong(3); + short inputIndex = rsetInput.getShort(4); + String inputType = rsetInput.getString(5); + boolean inputStrMask = rsetInput.getBoolean(6); + short inputStrLength = rsetInput.getShort(7); + + MInputType mit = MInputType.valueOf(inputType); + + MInput input = null; + switch (mit) { + case STRING: + input = new MStringInput(inputName, inputStrMask, inputStrLength); + break; + case MAP: + input = new MMapInput(inputName); + break; + default: + throw new SqoopException(DerbyRepoError.DERBYREPO_0006, + "input-" + inputName + ":" + inputId + ":" + + "form-" + inputForm + ":" + mit.name()); + } + input.setPersistenceId(inputId); + + if (mf.getInputs().size() != inputIndex) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0009, + "form: " + mf + "; input: " + input); + } + + mf.getInputs().add(input); + } + + if (mf.getInputs().size() == 0) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0008, + "connector-" + formConnectorId + ":" + mf); + } + + MFormType mft = MFormType.valueOf(formType); + switch (mft) { + case CONNECTION: + if (mc.getConnectionForms().size() != formIndex) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0010, + "connector: " + mc + "; form: " + mf); + } + mc.getConnectionForms().add(mf); + break; + case JOB: + if (mc.getConnectionForms().size() != formIndex) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0010, + "connector: " + mc + "; form: " + mf); + } + mc.getJobForms().add(mf); + break; + default: + throw new SqoopException(DerbyRepoError.DERBYREPO_0007, + "connector-" + formConnectorId + ":" + mf); + } + } + } catch (SQLException ex) { + throw new SqoopException(DerbyRepoError.DERBYREPO_0004, shortName, ex); + } finally { + if (baseConnectorFetchStmt != null) { + try { + baseConnectorFetchStmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close base connector fetch statement", ex); + } + } + if (formFetchStmt != null) { + try { + formFetchStmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close form fetch statement", ex); + } + } + if (inputFetchStmt != null) { + try { + inputFetchStmt.close(); + } catch (SQLException ex) { + LOG.error("Unable to close input fetch statement", ex); + } + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("Looking up connector: " + shortName + ", found: " + mc); + } + return mc; + } +} diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java new file mode 100644 index 00000000..83836cbb --- /dev/null +++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java @@ -0,0 +1,87 @@ +/** + * 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.repository.derby; + +public final class DerbySchemaConstants { + + public static final String SCHEMA_SQOOP = "SQOOP"; + + private static final String SCHEMA_PREFIX = SCHEMA_SQOOP + "."; + + public static final String TABLE_SQ_CONNECTOR_NAME = "SQ_CONNECTOR"; + + public static final String TABLE_SQ_CONNECTOR = SCHEMA_PREFIX + + TABLE_SQ_CONNECTOR_NAME; + + public static final String COLUMN_SQC_ID = "SQC_ID"; + + public static final String COLUMN_SQC_NAME = "SQC_NAME"; + + public static final String COLUMN_SQC_CLASS = "SQC_CLASS"; + + public static final String TABLE_SQ_FORM_NAME = "SQ_FORM"; + + public static final String TABLE_SQ_FORM = SCHEMA_PREFIX + + TABLE_SQ_FORM_NAME; + + public static final String COLUMN_SQF_ID = "SQF_ID"; + + public static final String COLUMN_SQF_CONNECTOR = "SQF_CONNECTOR"; + + public static final String COLUMN_SQF_NAME = "SQF_NAME"; + + public static final String COLUMN_SQF_TYPE = "SQF_TYPE"; + + public static final String COLUMN_SQF_INDEX = "SQF_INDEX"; + + + public static final String TABLE_SQ_INPUT_NAME = "SQ_INPUT"; + + public static final String TABLE_SQ_INPUT = SCHEMA_PREFIX + + TABLE_SQ_INPUT_NAME; + + public static final String COLUMN_SQI_ID = "SQI_ID"; + + public static final String COLUMN_SQI_NAME = "SQI_NAME"; + + public static final String COLUMN_SQI_FORM = "SQI_FORM"; + + public static final String COLUMN_SQI_INDEX = "SQI_INDEX"; + + public static final String COLUMN_SQI_TYPE = "SQI_TYPE"; + + public static final String COLUMN_SQI_STRMASK = "SQI_STRMASK"; + + public static final String COLUMN_SQI_STRLENGTH = "SQI_STRLENGTH"; + + + public static final String TABLE_SQ_STRING_NAME = "SQ_STRING"; + public static final String TABLE_SQ_STRING = SCHEMA_PREFIX + + TABLE_SQ_STRING_NAME; + + public static final String COLUMN_SQS_ID = "SQS_ID"; + + public static final String COLUMN_SQS_MASK = "SQS_MASK"; + + public static final String COLUMN_SQS_LENGTH = "SQS_LENGTH"; + + + private DerbySchemaConstants() { + // Disable explicit object creation + } +} 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 new file mode 100644 index 00000000..b393d96d --- /dev/null +++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java @@ -0,0 +1,146 @@ +/** + * 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.repository.derby; + +import static org.apache.sqoop.repository.derby.DerbySchemaConstants.*; + +/** + * DDL queries that create the Sqoop repository schema in Derby database. These + * queries create the following tables: + * + *

+ * SQ_CONNECTOR: Connector registration. + *

+ *    +----------------------------+
+ *    | SQ_CONNECTOR               |
+ *    +----------------------------+
+ *    | SQC_ID: BIGINT PK AUTO-GEN |
+ *    | SQC_NAME: VARCHAR(64)      |
+ *    | SQC_CLASS: VARCHAR(255)    |
+ *    +----------------------------+
+ * 
+ *

+ *

+ * SQ_FORM: Form details. + *

+ *    +-----------------------------+
+ *    | SQ_FORM                     |
+ *    +-----------------------------+
+ *    | SQF_ID: BIGINT PK AUTO-GEN  |
+ *    | SQF_CONNECTOR: BIGINT       | FK SQ_CONNECTOR(SQC_ID)
+ *    | SQF_NAME: VARCHAR(64)       |
+ *    | SQF_TYPE: VARCHAR(32)       | "CONNECTION"|"JOB"
+ *    | SQF_INDEX: SMALLINT         |
+ *    +-----------------------------+
+ * 
+ *

+ *

+ * SQ_INPUT: Input details + *

+ *    +----------------------------+
+ *    | SQ_INPUT                   |
+ *    +----------------------------+
+ *    | SQI_ID: BIGINT PK AUTO-GEN |
+ *    | SQI_NAME: VARCHAR(64)      |
+ *    | SQI_FORM: BIGINT           | FK SQ_FORM(SQF_ID)
+ *    | SQI_INDEX: SMALLINT        |
+ *    | SQI_TYPE: VARCHAR(32)      | "STRING"|"MAP"
+ *    | SQI_STRMASK: BOOLEAN       |
+ *    | SQI_STRLENGTH: SMALLINT    |
+ *    +----------------------------+
+ * 
+ *

+ */ +public final class DerbySchemaQuery { + + // DDL: Create schema + public static final String QUERY_CREATE_SCHEMA_SQOOP = + "CREATE SCHEMA " + SCHEMA_SQOOP; + + // DDL: Create table SQ_CONNECTOR + public static final String QUERY_CREATE_TABLE_SQ_CONNECTOR = + "CREATE TABLE " + TABLE_SQ_CONNECTOR + " (" + COLUMN_SQC_ID + + " BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) " + + "PRIMARY KEY, " + COLUMN_SQC_NAME + " VARCHAR(64), " + COLUMN_SQC_CLASS + + " VARCHAR(255))"; + + // DDL: Create table SQ_FORM + public static final String QUERY_CREATE_TABLE_SQ_FORM = + "CREATE TABLE " + TABLE_SQ_FORM + " (" + COLUMN_SQF_ID + + " BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) " + + "PRIMARY KEY, " + COLUMN_SQF_CONNECTOR + " BIGINT, " + + COLUMN_SQF_NAME + " VARCHAR(64), " + COLUMN_SQF_TYPE + " VARCHAR(32), " + + COLUMN_SQF_INDEX + " SMALLINT, " + " FOREIGN KEY (" + + COLUMN_SQF_CONNECTOR+ ") REFERENCES " + TABLE_SQ_CONNECTOR + " (" + + COLUMN_SQC_ID + "))"; + + + // DDL: Create table SQ_INPUT + public static final String QUERY_CREATE_TABLE_SQ_INPUT = + "CREATE TABLE " + TABLE_SQ_INPUT + " (" + COLUMN_SQI_ID + + " BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) " + + "PRIMARY KEY, " + COLUMN_SQI_NAME + " VARCHAR(64), " + + COLUMN_SQI_FORM + " BIGINT, " + COLUMN_SQI_INDEX + " SMALLINT, " + + COLUMN_SQI_TYPE + " VARCHAR(32), " + COLUMN_SQI_STRMASK + " BOOLEAN, " + + COLUMN_SQI_STRLENGTH + " SMALLINT, FOREIGN KEY (" + COLUMN_SQI_FORM + + ") REFERENCES " + TABLE_SQ_FORM + " (" + COLUMN_SQF_ID + "))"; + + // DML: Fetch connector Given Name + public static final String STMT_FETCH_BASE_CONNECTOR = + "SELECT " + COLUMN_SQC_ID + ", " + COLUMN_SQC_NAME + ", " + + COLUMN_SQC_CLASS + " FROM " + TABLE_SQ_CONNECTOR + " WHERE " + + COLUMN_SQC_NAME + " = ?"; + + + // DML: Fetch all forms for a given connector + public static final String STMT_FETCH_FORM = + "SELECT " + COLUMN_SQF_ID + ", " + COLUMN_SQF_CONNECTOR + ", " + + COLUMN_SQF_NAME + ", " + COLUMN_SQF_TYPE + ", " + COLUMN_SQF_INDEX + + " FROM " + TABLE_SQ_FORM + " WHERE " + COLUMN_SQF_CONNECTOR + + " = ? ORDER BY " + COLUMN_SQF_INDEX; + + // DML: Fetch inputs for a given form + 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_INDEX; + + // DML: Insert connector base + public static final String STMT_INSERT_CONNECTOR_BASE = + "INSERT INTO " + TABLE_SQ_CONNECTOR + " (" + COLUMN_SQC_NAME + + ", " + COLUMN_SQC_CLASS + ") VALUES ( ?, ?)"; + + // DML: Insert form base + public static final String STMT_INSERT_FORM_BASE = + "INSERT INTO " + TABLE_SQ_FORM + " (" + COLUMN_SQF_CONNECTOR + + ", " + COLUMN_SQF_NAME + ", " + COLUMN_SQF_TYPE + ", " + + COLUMN_SQF_INDEX + ") VALUES ( ?, ?, ?, ?)"; + + // DML: Insert form input + public static final String STMT_INSERT_INPUT_BASE = + "INSERT INTO " + TABLE_SQ_INPUT + " (" + COLUMN_SQI_NAME + ", " + + COLUMN_SQI_FORM + ", " + COLUMN_SQI_INDEX + ", " + COLUMN_SQI_TYPE + + ", " + COLUMN_SQI_STRMASK + ", " + COLUMN_SQI_STRLENGTH + ") " + + "VALUES (?, ?, ?, ?, ?, ?)"; + + private DerbySchemaQuery() { + // Disable explicit object creation + } +} diff --git a/server/pom.xml b/server/pom.xml new file mode 100644 index 00000000..91c37c41 --- /dev/null +++ b/server/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-server + Sqoop Server + war + + + + org.apache.sqoop + sqoop-core + 2.0.0-SNAPSHOT + + + + org.apache.sqoop + sqoop-common + 2.0.0-SNAPSHOT + + + + org.apache.sqoop.repository + sqoop-repository-derby + 2.0.0-SNAPSHOT + + + + org.apache.sqoop.connector + sqoop-connector-generic-jdbc + 2.0.0-SNAPSHOT + + + + org.apache.sqoop.connector + sqoop-connector-mysql-jdbc + 2.0.0-SNAPSHOT + + + + javax.servlet + servlet-api + provided + + + + com.googlecode.json-simple + json-simple + + + + + sqoop + + + diff --git a/server/src/main/java/org/apache/sqoop/handler/VersionRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/VersionRequestHandler.java new file mode 100644 index 00000000..75c36d81 --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/handler/VersionRequestHandler.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.handler; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.VersionInfo; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.json.JsonBean; +import org.apache.sqoop.json.VersionBean; +import org.apache.sqoop.server.RequestContext; +import org.apache.sqoop.server.RequestHandler; + +public class VersionRequestHandler implements RequestHandler { + + private static final Logger LOG = + Logger.getLogger(VersionRequestHandler.class); + + /** The API version supported by this server */ + public static final String PROTOCOL_V1 = "1"; + + + private final VersionBean versionBean; + + public VersionRequestHandler() { + String[] protocols = { PROTOCOL_V1 }; + versionBean = new VersionBean(VersionInfo.getVersion(), + VersionInfo.getRevision(), VersionInfo.getDate(), + VersionInfo.getUser(), VersionInfo.getUrl(), protocols); + + LOG.info("VersionRequestHandler initialized"); + } + + + @Override + public JsonBean handleEvent(RequestContext ctx) throws SqoopException { + return versionBean; + } +} diff --git a/server/src/main/java/org/apache/sqoop/server/RequestContext.java b/server/src/main/java/org/apache/sqoop/server/RequestContext.java new file mode 100644 index 00000000..b1ea853b --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/server/RequestContext.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.server; + +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +public class RequestContext { + + private final HttpServletRequest request; + private final HttpServletResponse response; + + public RequestContext(HttpServletRequest req, HttpServletResponse resp) { + request = req; + response = resp; + } + + public HttpServletRequest getRequest() { + return request; + } + + public HttpServletResponse getResponse() { + return response; + } + + public String getPath() { + return request.getRequestURL().toString(); + } +} diff --git a/server/src/main/java/org/apache/sqoop/server/RequestHandler.java b/server/src/main/java/org/apache/sqoop/server/RequestHandler.java new file mode 100644 index 00000000..43d98c51 --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/server/RequestHandler.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.server; + + +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.json.JsonBean; + +public interface RequestHandler { + + public JsonBean handleEvent(RequestContext ctx) throws SqoopException; +} diff --git a/server/src/main/java/org/apache/sqoop/server/ServerInitializer.java b/server/src/main/java/org/apache/sqoop/server/ServerInitializer.java new file mode 100644 index 00000000..6519f522 --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/server/ServerInitializer.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.server; + +import javax.servlet.ServletContextEvent; +import javax.servlet.ServletContextListener; + +import org.apache.log4j.Logger; +import org.apache.sqoop.common.SqoopException; +import org.apache.sqoop.connector.ConnectorManager; +import org.apache.sqoop.core.SqoopConfiguration; +import org.apache.sqoop.repository.RepositoryManager; + + +/** + * Initializes the Sqoop server. This listener is also responsible for + * cleaning up any resources occupied by the server during the system shutdown. + */ +public class ServerInitializer implements ServletContextListener { + + private static final Logger LOG = + Logger.getLogger(ServerInitializer.class); + + public void contextDestroyed(ServletContextEvent arg0) { + ConnectorManager.destroy(); + RepositoryManager.destroy(); + SqoopConfiguration.destroy(); + } + + public void contextInitialized(ServletContextEvent arg0) { + try { + SqoopConfiguration.initialize(); + RepositoryManager.initialize(); + ConnectorManager.initialize(); + } catch (RuntimeException ex) { + LOG.error("Server startup failure", ex); + throw ex; + } + } +} diff --git a/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java b/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java new file mode 100644 index 00000000..f36aa057 --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/server/SqoopProtocolServlet.java @@ -0,0 +1,143 @@ +/** + * 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; + +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.Charset; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +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.common.SqoopException; +import org.apache.sqoop.common.SqoopProtocolConstants; +import org.apache.sqoop.common.SqoopResponseCode; +import org.apache.sqoop.core.CoreError; +import org.apache.sqoop.json.JsonBean; + +@SuppressWarnings("serial") +public class SqoopProtocolServlet extends HttpServlet { + + private static final Logger LOG = + Logger.getLogger(SqoopProtocolServlet.class); + + @Override + protected final void doGet(HttpServletRequest req, HttpServletResponse resp) + throws ServletException, IOException { + RequestContext rctx = new RequestContext(req, resp); + + try { + JsonBean bean = handleGetRequest(rctx); + if (bean != null) { + sendSuccessResponse(rctx, bean); + } + } catch (Exception ex) { + LOG.error("Exception in GET " + rctx.getPath(), ex); + sendErrorResponse(rctx, ex); + } + } + + @Override + protected final void doPost(HttpServletRequest req, HttpServletResponse resp) + throws ServletException, IOException { + RequestContext rctx = new RequestContext(req, resp); + try { + JsonBean bean = handlePostRequest(rctx); + if (bean != null) { + sendSuccessResponse(rctx, bean); + } + } catch (Exception ex) { + LOG.error("Exception in POST " + rctx.getPath(), ex); + sendErrorResponse(rctx, ex); + } + } + + private void sendSuccessResponse(RequestContext ctx, JsonBean bean) + throws IOException { + HttpServletResponse response = ctx.getResponse(); + response.setStatus(HttpServletResponse.SC_OK); + setContentType(response); + setHeaders(response, SqoopResponseCode.SQOOP_1000); + String responseString = bean.extract().toJSONString(); + response.getWriter().write(responseString); + response.getWriter().flush(); + } + + private void sendErrorResponse(RequestContext ctx, Exception ex) + throws IOException + { + HttpServletResponse response = ctx.getResponse(); + setContentType(response); + setHeaders(response, SqoopResponseCode.SQOOP_2000); + + if (ex != null) { + ErrorCode ec = null; + if (ex instanceof SqoopException) { + ec = ((SqoopException) ex).getErrorCode(); + } else { + ec = CoreError.CORE_0000; + } + + response.setHeader( + SqoopProtocolConstants.HEADER_SQOOP_INTERNAL_ERROR_CODE, + ec.getCode()); + + response.setHeader( + SqoopProtocolConstants.HEADER_SQOOP_INTERNAL_ERROR_MESSAGE, + ex.getMessage()); + + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, + new ExceptionInfo(ec.getCode(), + ex.getMessage(), ex).extract().toJSONString()); + } else { + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + } + + } + + private void setContentType(HttpServletResponse response) { + response.setContentType(SqoopProtocolConstants.JSON_CONTENT_TYPE); + } + + private void setHeaders(HttpServletResponse response, SqoopResponseCode code) + { + response.setHeader(SqoopProtocolConstants.HEADER_SQOOP_ERROR_CODE, + code.getCode()); + response.setHeader(SqoopProtocolConstants.HEADER_SQOOP_ERROR_MESSAGE, + code.getMessage()); + + + } + + protected JsonBean handleGetRequest(RequestContext ctx) throws Exception { + super.doGet(ctx.getRequest(), ctx.getResponse()); + + return null; + } + + protected JsonBean handlePostRequest(RequestContext ctx) throws Exception { + super.doPost(ctx.getRequest(), ctx.getResponse()); + + return null; + } +} diff --git a/server/src/main/java/org/apache/sqoop/server/VersionServlet.java b/server/src/main/java/org/apache/sqoop/server/VersionServlet.java new file mode 100644 index 00000000..a62c9ce9 --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/server/VersionServlet.java @@ -0,0 +1,40 @@ +/** + * 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; + +import org.apache.sqoop.handler.VersionRequestHandler; +import org.apache.sqoop.json.JsonBean; + +/** + * Exposes the supported versions available in the server. + * + */ +@SuppressWarnings("serial") +public class VersionServlet extends SqoopProtocolServlet { + + private RequestHandler versionRequestHandler; + + public VersionServlet() { + versionRequestHandler = new VersionRequestHandler(); + } + + @Override + protected JsonBean handleGetRequest(RequestContext ctx) throws Exception { + return versionRequestHandler.handleEvent(ctx); + } +} diff --git a/server/src/main/java/org/apache/sqoop/server/v1/ConnectorServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/ConnectorServlet.java new file mode 100644 index 00000000..07a86ae2 --- /dev/null +++ b/server/src/main/java/org/apache/sqoop/server/v1/ConnectorServlet.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.server.v1; + +import org.apache.sqoop.json.JsonBean; +import org.apache.sqoop.server.RequestContext; +import org.apache.sqoop.server.SqoopProtocolServlet; + +/** + * Displays the list of connectors that are available in the system via + * a GET request. + */ +@SuppressWarnings("serial") +public class ConnectorServlet extends SqoopProtocolServlet { + + @Override + protected JsonBean handleGetRequest(RequestContext ctx) throws Exception { + // TODO Auto-generated method stub + return super.handleGetRequest(ctx); + } +} diff --git a/server/src/main/webapp/WEB-INF/web.xml b/server/src/main/webapp/WEB-INF/web.xml new file mode 100644 index 00000000..17238700 --- /dev/null +++ b/server/src/main/webapp/WEB-INF/web.xml @@ -0,0 +1,42 @@ + + + + + Apache Sqoop + + + + org.apache.sqoop.server.ServerInitializer + + + + VersionServlet + org.apache.sqoop.server.VersionServlet + 1 + + + + VersionServlet + /version + + + + diff --git a/server/src/main/webapp/index.html b/server/src/main/webapp/index.html new file mode 100644 index 00000000..06513dd4 --- /dev/null +++ b/server/src/main/webapp/index.html @@ -0,0 +1,23 @@ + + + +Apache Sqoop + +Apache Sqoop + + diff --git a/spi/pom.xml b/spi/pom.xml new file mode 100644 index 00000000..f71145aa --- /dev/null +++ b/spi/pom.xml @@ -0,0 +1,41 @@ + + + + + 4.0.0 + + + org.apache + sqoop + 2.0.0-SNAPSHOT + + + org.apache.sqoop + sqoop-spi + Sqoop SPI + + + + org.apache.sqoop + sqoop-common + 2.0.0-SNAPSHOT + + + diff --git a/spi/src/main/java/org/apache/sqoop/connector/spi/SqoopConnector.java b/spi/src/main/java/org/apache/sqoop/connector/spi/SqoopConnector.java new file mode 100644 index 00000000..43d7344a --- /dev/null +++ b/spi/src/main/java/org/apache/sqoop/connector/spi/SqoopConnector.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.sqoop.connector.spi; + +import java.util.List; +import java.util.Locale; +import java.util.ResourceBundle; + +import org.apache.sqoop.model.MForm; + +/** + * Service provider interface for Sqoop Connectors. + */ +public interface SqoopConnector { + + /** + * @param locale + * @return the resource bundle associated with the given locale. + */ + public ResourceBundle getBundle(Locale locale); + + /** + * @return a list of MForm that provide metadata about input needed + * by Sqoop to create a connection object using this connector. + */ + public List getConnectionForms(); + + + /** + * @return a list of MForm that provide metadata about input needed + * by Sqoop to create a job object using this connector. + */ + public List getJobForms(); +}