зеркало из https://github.com/github/sqoop.git
SQOOP-1497: Sqoop2: Entity Nomenclature Revisited
This commit is contained in:
Родитель
b04e796f01
Коммит
049994a021
|
@ -17,16 +17,16 @@
|
|||
*/
|
||||
package org.apache.sqoop.client;
|
||||
|
||||
import org.apache.sqoop.client.request.SqoopRequests;
|
||||
import org.apache.sqoop.client.request.SqoopResourceRequests;
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.json.ConnectorBean;
|
||||
import org.apache.sqoop.json.FrameworkBean;
|
||||
import org.apache.sqoop.json.DriverConfigBean;
|
||||
import org.apache.sqoop.json.ValidationResultBean;
|
||||
import org.apache.sqoop.model.FormUtils;
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.apache.sqoop.model.MConnector;
|
||||
import org.apache.sqoop.model.MFramework;
|
||||
import org.apache.sqoop.model.MDriverConfig;
|
||||
import org.apache.sqoop.model.MJob;
|
||||
import org.apache.sqoop.model.MSubmission;
|
||||
import org.apache.sqoop.validation.Status;
|
||||
|
@ -45,7 +45,7 @@ import java.util.ResourceBundle;
|
|||
* implementation is not thread safe.
|
||||
*
|
||||
* SqoopClient is keeping cache of objects that are unlikely to be changed
|
||||
* (Resources, Connector structures). Volatile structures (Connections, Jobs)
|
||||
* (Resources, Connector structures). Volatile structures (Links, Jobs)
|
||||
* are not cached.
|
||||
*/
|
||||
public class SqoopClient {
|
||||
|
@ -53,32 +53,29 @@ public class SqoopClient {
|
|||
/**
|
||||
* Underlying request object to fetch data from Sqoop server.
|
||||
*/
|
||||
private SqoopRequests requests;
|
||||
private SqoopResourceRequests resourceRequests;
|
||||
|
||||
/**
|
||||
* True if user retrieved all connectors at once.
|
||||
*/
|
||||
private boolean allConnectors;
|
||||
|
||||
/**
|
||||
* All cached bundles for all connectors.
|
||||
*/
|
||||
private Map<Long, ResourceBundle> bundles;
|
||||
|
||||
/**
|
||||
* Cached framework bundle.
|
||||
*/
|
||||
private ResourceBundle frameworkBundle;
|
||||
|
||||
private boolean isAllConnectors;
|
||||
/**
|
||||
* All cached connectors.
|
||||
*/
|
||||
private Map<Long, MConnector> connectors;
|
||||
/**
|
||||
* All cached bundles for all connectors.
|
||||
*/
|
||||
private Map<Long, ResourceBundle> connectorConfigBundles;
|
||||
|
||||
/**
|
||||
* Cached framework.
|
||||
* Cached driverConfig.
|
||||
*/
|
||||
private MFramework framework;
|
||||
private MDriverConfig driverConfig;
|
||||
/**
|
||||
* Cached driverConfig bundle.
|
||||
*/
|
||||
private ResourceBundle driverConfigBundle;
|
||||
|
||||
/**
|
||||
* Status flags used when updating the submission callback status
|
||||
|
@ -90,7 +87,7 @@ public class SqoopClient {
|
|||
}
|
||||
|
||||
public SqoopClient(String serverUrl) {
|
||||
requests = new SqoopRequests();
|
||||
resourceRequests = new SqoopResourceRequests();
|
||||
setServerUrl(serverUrl);
|
||||
}
|
||||
|
||||
|
@ -102,7 +99,7 @@ public class SqoopClient {
|
|||
* @param serverUrl Server URL
|
||||
*/
|
||||
public void setServerUrl(String serverUrl) {
|
||||
requests.setServerUrl(serverUrl);
|
||||
resourceRequests.setServerUrl(serverUrl);
|
||||
clearCache();
|
||||
}
|
||||
|
||||
|
@ -111,8 +108,8 @@ public class SqoopClient {
|
|||
*
|
||||
* @param requests SqoopRequests object
|
||||
*/
|
||||
public void setSqoopRequests(SqoopRequests requests) {
|
||||
this.requests = requests;
|
||||
public void setSqoopRequests(SqoopResourceRequests requests) {
|
||||
this.resourceRequests = requests;
|
||||
clearCache();
|
||||
}
|
||||
|
||||
|
@ -120,11 +117,11 @@ public class SqoopClient {
|
|||
* Clear internal cache.
|
||||
*/
|
||||
public void clearCache() {
|
||||
bundles = new HashMap<Long, ResourceBundle>();
|
||||
frameworkBundle = null;
|
||||
connectorConfigBundles = new HashMap<Long, ResourceBundle>();
|
||||
driverConfigBundle = null;
|
||||
connectors = new HashMap<Long, MConnector>();
|
||||
framework = null;
|
||||
allConnectors = false;
|
||||
driverConfig = null;
|
||||
isAllConnectors = false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -155,7 +152,7 @@ public class SqoopClient {
|
|||
|
||||
// If the connector wasn't in cache and we have all connectors,
|
||||
// it simply do not exists.
|
||||
if(allConnectors) return null;
|
||||
if(isAllConnectors) return null;
|
||||
|
||||
// Retrieve all connectors from server
|
||||
getConnectors();
|
||||
|
@ -186,9 +183,9 @@ public class SqoopClient {
|
|||
* @param cid Connector id
|
||||
*/
|
||||
private void retrieveConnector(long cid) {
|
||||
ConnectorBean request = requests.readConnector(cid);
|
||||
ConnectorBean request = resourceRequests.readConnector(cid);
|
||||
connectors.put(cid, request.getConnectors().get(0));
|
||||
bundles.put(cid, request.getResourceBundles().get(cid));
|
||||
connectorConfigBundles.put(cid, request.getResourceBundles().get(cid));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -197,16 +194,16 @@ public class SqoopClient {
|
|||
* @return
|
||||
*/
|
||||
public Collection<MConnector> getConnectors() {
|
||||
if(allConnectors) {
|
||||
if(isAllConnectors) {
|
||||
return connectors.values();
|
||||
}
|
||||
|
||||
ConnectorBean bean = requests.readConnector(null);
|
||||
allConnectors = true;
|
||||
ConnectorBean bean = resourceRequests.readConnector(null);
|
||||
isAllConnectors = true;
|
||||
for(MConnector connector : bean.getConnectors()) {
|
||||
connectors.put(connector.getPersistenceId(), connector);
|
||||
}
|
||||
bundles = bean.getResourceBundles();
|
||||
connectorConfigBundles = bean.getResourceBundles();
|
||||
|
||||
return connectors.values();
|
||||
}
|
||||
|
@ -214,173 +211,171 @@ public class SqoopClient {
|
|||
/**
|
||||
* Get resource bundle for given connector.
|
||||
*
|
||||
* @param cid Connector id.
|
||||
* @param connectorId Connector id.
|
||||
* @return
|
||||
*/
|
||||
public ResourceBundle getResourceBundle(long cid) {
|
||||
if(bundles.containsKey(cid)) {
|
||||
return bundles.get(cid);
|
||||
public ResourceBundle getConnectorConfigResourceBundle(long connectorId) {
|
||||
if(connectorConfigBundles.containsKey(connectorId)) {
|
||||
return connectorConfigBundles.get(connectorId);
|
||||
}
|
||||
|
||||
retrieveConnector(cid);
|
||||
return bundles.get(cid);
|
||||
retrieveConnector(connectorId);
|
||||
return connectorConfigBundles.get(connectorId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return framework metadata.
|
||||
* Return driver config.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public MFramework getFramework() {
|
||||
if(framework != null) {
|
||||
return framework.clone(false);
|
||||
public MDriverConfig getDriverConfig() {
|
||||
if(driverConfig != null) {
|
||||
return driverConfig.clone(false);
|
||||
}
|
||||
|
||||
retrieveFramework();
|
||||
return framework.clone(false);
|
||||
retrieveAndCacheDriverConfig();
|
||||
return driverConfig.clone(false);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve framework structure and cache it.
|
||||
* Retrieve driverConfig and cache it.
|
||||
*/
|
||||
private void retrieveFramework() {
|
||||
FrameworkBean request = requests.readFramework();
|
||||
framework = request.getFramework();
|
||||
frameworkBundle = request.getResourceBundle();
|
||||
private void retrieveAndCacheDriverConfig() {
|
||||
DriverConfigBean driverConfigBean = resourceRequests.readDriverConfig();
|
||||
driverConfig = driverConfigBean.getDriverConfig();
|
||||
driverConfigBundle = driverConfigBean.getResourceBundle();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return framework bundle.
|
||||
* Return driverConfig bundle.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public ResourceBundle getFrameworkResourceBundle() {
|
||||
if(frameworkBundle != null) {
|
||||
return frameworkBundle;
|
||||
public ResourceBundle getDriverConfigBundle() {
|
||||
if(driverConfigBundle != null) {
|
||||
return driverConfigBundle;
|
||||
}
|
||||
|
||||
retrieveFramework();
|
||||
return frameworkBundle;
|
||||
retrieveAndCacheDriverConfig();
|
||||
return driverConfigBundle;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new connection object for given connector.
|
||||
* Create new link object for given connector id
|
||||
*
|
||||
* @param cid Connector id
|
||||
* @param connectorId Connector id
|
||||
* @return
|
||||
*/
|
||||
public MConnection newConnection(long cid) {
|
||||
return new MConnection(
|
||||
cid,
|
||||
getConnector(cid).getConnectionForms(),
|
||||
getFramework().getConnectionForms()
|
||||
public MLink createLink(long connectorId) {
|
||||
return new MLink(
|
||||
connectorId,
|
||||
getConnector(connectorId).getConnectionForms(),
|
||||
getDriverConfig().getConnectionForms()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new connection object for given connector.
|
||||
* Create new link object for given connector name
|
||||
*
|
||||
* @param connectorName Connector name
|
||||
* @return
|
||||
*/
|
||||
public MConnection newConnection(String connectorName) {
|
||||
public MLink createLink(String connectorName) {
|
||||
MConnector connector = getConnector(connectorName);
|
||||
if(connector == null) {
|
||||
throw new SqoopException(ClientError.CLIENT_0003, connectorName);
|
||||
}
|
||||
|
||||
return newConnection(connector.getPersistenceId());
|
||||
return createLink(connector.getPersistenceId());
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve connection with given id.
|
||||
* Retrieve link for given id.
|
||||
*
|
||||
* @param xid Connnection id
|
||||
* @param linkId Link id
|
||||
* @return
|
||||
*/
|
||||
public MConnection getConnection(long xid) {
|
||||
return requests.readConnection(xid).getConnections().get(0);
|
||||
public MLink getLink(long linkId) {
|
||||
return resourceRequests.readLink(linkId).getLinks().get(0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve list of all connections.
|
||||
* Retrieve list of all links.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public List<MConnection> getConnections() {
|
||||
return requests.readConnection(null).getConnections();
|
||||
public List<MLink> getLinks() {
|
||||
return resourceRequests.readLink(null).getLinks();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the connection on server.
|
||||
* Create the link and save to the repository
|
||||
*
|
||||
* @param connection Connection that should be created
|
||||
* @param link link that should be created
|
||||
* @return
|
||||
*/
|
||||
public Status createConnection(MConnection connection) {
|
||||
return applyValidations(requests.createConnection(connection), connection);
|
||||
public Status saveLink(MLink link) {
|
||||
return applyLinkValidations(resourceRequests.saveLink(link), link);
|
||||
}
|
||||
|
||||
/**
|
||||
* Update connection on the server.
|
||||
* Update link on the server.
|
||||
*
|
||||
* @param connection Connection that should be updated
|
||||
* @param link link that should be updated
|
||||
* @return
|
||||
*/
|
||||
public Status updateConnection(MConnection connection) {
|
||||
return applyValidations(requests.updateConnection(connection), connection);
|
||||
public Status updateLink(MLink link) {
|
||||
return applyLinkValidations(resourceRequests.updateLink(link), link);
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable/disable connection with given id
|
||||
* Enable/disable link with given id
|
||||
*
|
||||
* @param xid Connection id
|
||||
* @param linkId link id
|
||||
* @param enabled Enable or disable
|
||||
*/
|
||||
public void enableConnection(long xid, boolean enabled) {
|
||||
requests.enableConnection(xid, enabled);
|
||||
public void enableLink(long linkId, boolean enabled) {
|
||||
resourceRequests.enableLink(linkId, enabled);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete connection with given id.
|
||||
* Delete link with given id.
|
||||
*
|
||||
* @param xid Connection id
|
||||
* @param linkId link id
|
||||
*/
|
||||
public void deleteConnection(long xid) {
|
||||
requests.deleteConnection(xid);
|
||||
public void deleteLink(long linkId) {
|
||||
resourceRequests.deleteLink(linkId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new job the for given connections.
|
||||
* Create new job the for given links.
|
||||
*
|
||||
* @param fromXid From Connection id
|
||||
* @param toXid To Connection id
|
||||
* @param fromLinkId From link id
|
||||
* @param toLinkId To link id
|
||||
* @return
|
||||
*/
|
||||
public MJob newJob(long fromXid, long toXid) {
|
||||
MConnection fromConnection = getConnection(fromXid);
|
||||
MConnection toConnection = getConnection(toXid);
|
||||
public MJob createJob(long fromLinkId, long toLinkId) {
|
||||
MLink fromLink = getLink(fromLinkId);
|
||||
MLink toLink = getLink(toLinkId);
|
||||
|
||||
return new MJob(
|
||||
fromConnection.getConnectorId(),
|
||||
toConnection.getConnectorId(),
|
||||
fromConnection.getPersistenceId(),
|
||||
toConnection.getPersistenceId(),
|
||||
getConnector(fromConnection.getConnectorId()).getJobForms(Direction.FROM),
|
||||
getConnector(toConnection.getConnectorId()).getJobForms(Direction.TO),
|
||||
getFramework().getJobForms()
|
||||
fromLink.getConnectorId(),
|
||||
toLink.getConnectorId(),
|
||||
fromLink.getPersistenceId(),
|
||||
toLink.getPersistenceId(),
|
||||
getConnector(fromLink.getConnectorId()).getJobForms(Direction.FROM),
|
||||
getConnector(toLink.getConnectorId()).getJobForms(Direction.TO),
|
||||
getDriverConfig().getJobForms()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve job for given id.
|
||||
*
|
||||
* @param jid Job id
|
||||
* @param jobId Job id
|
||||
* @return
|
||||
*/
|
||||
public MJob getJob(long jid) {
|
||||
return requests.readJob(jid).getJobs().get(0);
|
||||
public MJob getJob(long jobId) {
|
||||
return resourceRequests.readJob(jobId).getJobs().get(0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -389,17 +384,17 @@ public class SqoopClient {
|
|||
* @return
|
||||
*/
|
||||
public List<MJob> getJobs() {
|
||||
return requests.readJob(null).getJobs();
|
||||
return resourceRequests.readJob(null).getJobs();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create job on server.
|
||||
* Create job on server and save to the repository
|
||||
*
|
||||
* @param job Job that should be created
|
||||
* @return
|
||||
*/
|
||||
public Status createJob(MJob job) {
|
||||
return applyValidations(requests.createJob(job), job);
|
||||
public Status saveJob(MJob job) {
|
||||
return applyJobValidations(resourceRequests.saveJob(job), job);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -408,7 +403,7 @@ public class SqoopClient {
|
|||
* @return
|
||||
*/
|
||||
public Status updateJob(MJob job) {
|
||||
return applyValidations(requests.updateJob(job), job);
|
||||
return applyJobValidations(resourceRequests.updateJob(job), job);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -418,44 +413,45 @@ public class SqoopClient {
|
|||
* @param enabled Enable or disable
|
||||
*/
|
||||
public void enableJob(long jid, boolean enabled) {
|
||||
requests.enableJob(jid, enabled);
|
||||
resourceRequests.enableJob(jid, enabled);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete job with given id.
|
||||
*
|
||||
* @param jid Job id
|
||||
* @param jobId Job id
|
||||
*/
|
||||
public void deleteJob(long jid) {
|
||||
requests.deleteJob(jid);
|
||||
public void deleteJob(long jobId) {
|
||||
resourceRequests.deleteJob(jobId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Start job with given id.
|
||||
*
|
||||
* @param jid Job id
|
||||
* @param jobId Job id
|
||||
* @return
|
||||
*/
|
||||
public MSubmission startSubmission(long jid) {
|
||||
return requests.createSubmission(jid).getSubmissions().get(0);
|
||||
public MSubmission startSubmission(long jobId) {
|
||||
return resourceRequests.createSubmission(jobId).getSubmissions().get(0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Method used for synchronous job submission.
|
||||
* Pass null to callback parameter if submission status is not required and after completion
|
||||
* job execution returns MSubmission which contains final status of submission.
|
||||
* @param jid - Job ID
|
||||
* @param jobId - Job ID
|
||||
* @param callback - User may set null if submission status is not required, else callback methods invoked
|
||||
* @param pollTime - Server poll time
|
||||
* @return MSubmission - Final status of job submission
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public MSubmission startSubmission(long jid, SubmissionCallback callback, long pollTime) throws InterruptedException {
|
||||
public MSubmission startSubmission(long jobId, SubmissionCallback callback, long pollTime)
|
||||
throws InterruptedException {
|
||||
if(pollTime <= 0) {
|
||||
throw new SqoopException(ClientError.CLIENT_0002);
|
||||
}
|
||||
boolean first = true;
|
||||
MSubmission submission = requests.createSubmission(jid).getSubmissions().get(0);
|
||||
MSubmission submission = resourceRequests.createSubmission(jobId).getSubmissions().get(0);
|
||||
while(submission.getStatus().isRunning()) {
|
||||
if(first) {
|
||||
submissionCallback(callback, submission, SubmissionStatus.SUBMITTED);
|
||||
|
@ -464,7 +460,7 @@ public class SqoopClient {
|
|||
submissionCallback(callback, submission, SubmissionStatus.UPDATED);
|
||||
}
|
||||
Thread.sleep(pollTime);
|
||||
submission = getSubmissionStatus(jid);
|
||||
submission = getSubmissionStatus(jobId);
|
||||
}
|
||||
submissionCallback(callback, submission, SubmissionStatus.FINISHED);
|
||||
return submission;
|
||||
|
@ -477,9 +473,9 @@ public class SqoopClient {
|
|||
* @param submission
|
||||
* @param status
|
||||
*/
|
||||
private void submissionCallback(SubmissionCallback callback,
|
||||
MSubmission submission, SubmissionStatus status) {
|
||||
if(callback == null) {
|
||||
private void submissionCallback(SubmissionCallback callback, MSubmission submission,
|
||||
SubmissionStatus status) {
|
||||
if (callback == null) {
|
||||
return;
|
||||
}
|
||||
switch (status) {
|
||||
|
@ -501,7 +497,7 @@ public class SqoopClient {
|
|||
* @return
|
||||
*/
|
||||
public MSubmission stopSubmission(long jid) {
|
||||
return requests.deleteSubmission(jid).getSubmissions().get(0);
|
||||
return resourceRequests.deleteSubmission(jid).getSubmissions().get(0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -511,7 +507,7 @@ public class SqoopClient {
|
|||
* @return
|
||||
*/
|
||||
public MSubmission getSubmissionStatus(long jid) {
|
||||
return requests.readSubmission(jid).getSubmissions().get(0);
|
||||
return resourceRequests.readSubmission(jid).getSubmissions().get(0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -520,46 +516,46 @@ public class SqoopClient {
|
|||
* @return
|
||||
*/
|
||||
public List<MSubmission> getSubmissions() {
|
||||
return requests.readHistory(null).getSubmissions();
|
||||
return resourceRequests.readHistory(null).getSubmissions();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve list of submissions for given jobId.
|
||||
*
|
||||
* @param jid Job id
|
||||
* @param jobId Job id
|
||||
* @return
|
||||
*/
|
||||
public List<MSubmission> getSubmissionsForJob(long jid) {
|
||||
return requests.readHistory(jid).getSubmissions();
|
||||
public List<MSubmission> getSubmissionsForJob(long jobId) {
|
||||
return resourceRequests.readHistory(jobId).getSubmissions();
|
||||
}
|
||||
|
||||
private Status applyValidations(ValidationResultBean bean, MConnection connection) {
|
||||
private Status applyLinkValidations(ValidationResultBean bean, MLink link) {
|
||||
ValidationResult connector = bean.getValidationResults()[0];
|
||||
ValidationResult framework = bean.getValidationResults()[1];
|
||||
ValidationResult driverConfig = bean.getValidationResults()[1];
|
||||
|
||||
// Apply validation results
|
||||
FormUtils.applyValidation(connection.getConnectorPart().getForms(), connector);
|
||||
FormUtils.applyValidation(connection.getFrameworkPart().getForms(), framework);
|
||||
FormUtils.applyValidation(link.getConnectorPart().getForms(), connector);
|
||||
FormUtils.applyValidation(link.getFrameworkPart().getForms(), driverConfig);
|
||||
|
||||
Long id = bean.getId();
|
||||
if(id != null) {
|
||||
connection.setPersistenceId(id);
|
||||
link.setPersistenceId(id);
|
||||
}
|
||||
|
||||
return Status.getWorstStatus(connector.getStatus(), framework.getStatus());
|
||||
return Status.getWorstStatus(connector.getStatus(), driverConfig.getStatus());
|
||||
}
|
||||
|
||||
private Status applyValidations(ValidationResultBean bean, MJob job) {
|
||||
private Status applyJobValidations(ValidationResultBean bean, MJob job) {
|
||||
ValidationResult fromConnector = bean.getValidationResults()[0];
|
||||
ValidationResult toConnector = bean.getValidationResults()[1];
|
||||
ValidationResult framework = bean.getValidationResults()[2];
|
||||
ValidationResult driverConfig = bean.getValidationResults()[2];
|
||||
|
||||
// Apply validation results
|
||||
// @TODO(Abe): From/To validation.
|
||||
FormUtils.applyValidation(
|
||||
job.getConnectorPart(Direction.FROM).getForms(),
|
||||
fromConnector);
|
||||
FormUtils.applyValidation(job.getFrameworkPart().getForms(), framework);
|
||||
FormUtils.applyValidation(job.getFrameworkPart().getForms(), driverConfig);
|
||||
FormUtils.applyValidation(
|
||||
job.getConnectorPart(Direction.TO).getForms(),
|
||||
toConnector);
|
||||
|
@ -569,6 +565,6 @@ public class SqoopClient {
|
|||
job.setPersistenceId(id);
|
||||
}
|
||||
|
||||
return Status.getWorstStatus(fromConnector.getStatus(), framework.getStatus(), toConnector.getStatus());
|
||||
return Status.getWorstStatus(fromConnector.getStatus(), driverConfig.getStatus(), toConnector.getStatus());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.json.simple.JSONValue;
|
|||
* is supported as creation, update and delete might be done only directly on
|
||||
* server side.
|
||||
*/
|
||||
public class ConnectorRequest extends Request
|
||||
public class ConnectorResourceRequest extends ResourceRequest
|
||||
{
|
||||
public static final String RESOURCE = "v1/connector/";
|
||||
|
|
@ -17,26 +17,26 @@
|
|||
*/
|
||||
package org.apache.sqoop.client.request;
|
||||
|
||||
import org.apache.sqoop.json.FrameworkBean;
|
||||
import org.apache.sqoop.json.DriverConfigBean;
|
||||
import org.json.simple.JSONObject;
|
||||
import org.json.simple.JSONValue;
|
||||
|
||||
/**
|
||||
* Provide cRud semantics over RESTfull HTTP API for framework. Only read
|
||||
* Provide cRud semantics over RESTfull HTTP API for driverConfig. Only read
|
||||
* is supported as creation, update and delete is not allowed.
|
||||
*/
|
||||
public class FrameworkRequest extends Request {
|
||||
public class DriverConfigResourceRequest extends ResourceRequest {
|
||||
|
||||
public static final String RESOURCE = "v1/framework";
|
||||
public static final String RESOURCE = "v1/config/driver";
|
||||
|
||||
public FrameworkBean read(String serverUrl) {
|
||||
public DriverConfigBean read(String serverUrl) {
|
||||
String response = null;
|
||||
response = super.get(serverUrl + RESOURCE);
|
||||
JSONObject jsonObject = (JSONObject) JSONValue.parse(response);
|
||||
|
||||
FrameworkBean frameworkBean = new FrameworkBean();
|
||||
frameworkBean.restore(jsonObject);
|
||||
DriverConfigBean driverConfigBean = new DriverConfigBean();
|
||||
driverConfigBean.restore(jsonObject);
|
||||
|
||||
return frameworkBean;
|
||||
return driverConfigBean;
|
||||
}
|
||||
}
|
|
@ -27,7 +27,7 @@ import org.json.simple.JSONValue;
|
|||
* Provide CRUD semantics over RESTfull HTTP API for jobs. All operations
|
||||
* are normally supported.
|
||||
*/
|
||||
public class JobRequest extends Request {
|
||||
public class JobResourceRequest extends ResourceRequest {
|
||||
|
||||
public static final String RESOURCE = "v1/job/";
|
||||
|
|
@ -17,24 +17,24 @@
|
|||
*/
|
||||
package org.apache.sqoop.client.request;
|
||||
|
||||
import org.apache.sqoop.json.ConnectionBean;
|
||||
import org.apache.sqoop.json.LinkBean;
|
||||
import org.apache.sqoop.json.ValidationResultBean;
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.json.simple.JSONObject;
|
||||
import org.json.simple.JSONValue;
|
||||
|
||||
/**
|
||||
* Provide CRUD semantics over RESTfull HTTP API for connections. All operations
|
||||
* Provide CRUD semantics over RESTfull HTTP API for links. All operations
|
||||
* are normally supported.
|
||||
*/
|
||||
public class ConnectionRequest extends Request {
|
||||
public class LinkResourceRequest extends ResourceRequest {
|
||||
|
||||
public static final String RESOURCE = "v1/connection/";
|
||||
public static final String RESOURCE = "v1/link/";
|
||||
|
||||
private static final String ENABLE = "/enable";
|
||||
private static final String DISABLE = "/disable";
|
||||
|
||||
public ConnectionBean read(String serverUrl, Long xid) {
|
||||
public LinkBean read(String serverUrl, Long xid) {
|
||||
String response;
|
||||
if (xid == null) {
|
||||
response = super.get(serverUrl + RESOURCE + "all");
|
||||
|
@ -42,20 +42,18 @@ public class ConnectionRequest extends Request {
|
|||
response = super.get(serverUrl + RESOURCE + xid);
|
||||
}
|
||||
JSONObject jsonObject = (JSONObject)JSONValue.parse(response);
|
||||
|
||||
ConnectionBean connectionBean = new ConnectionBean();
|
||||
connectionBean.restore(jsonObject);
|
||||
|
||||
return connectionBean;
|
||||
LinkBean linkBean = new LinkBean();
|
||||
linkBean.restore(jsonObject);
|
||||
return linkBean;
|
||||
}
|
||||
|
||||
public ValidationResultBean create(String serverUrl, MConnection connection) {
|
||||
ConnectionBean connectionBean = new ConnectionBean(connection);
|
||||
public ValidationResultBean create(String serverUrl, MLink link) {
|
||||
LinkBean linkBean = new LinkBean(link);
|
||||
|
||||
// Extract all form inputs including sensitive inputs
|
||||
JSONObject connectionJson = connectionBean.extract(false);
|
||||
JSONObject linkJson = linkBean.extract(false);
|
||||
|
||||
String response = super.post(serverUrl + RESOURCE, connectionJson.toJSONString());
|
||||
String response = super.post(serverUrl + RESOURCE, linkJson.toJSONString());
|
||||
|
||||
ValidationResultBean validationBean = new ValidationResultBean();
|
||||
validationBean.restore((JSONObject) JSONValue.parse(response));
|
||||
|
@ -63,13 +61,13 @@ public class ConnectionRequest extends Request {
|
|||
return validationBean;
|
||||
}
|
||||
|
||||
public ValidationResultBean update(String serverUrl, MConnection connection) {
|
||||
ConnectionBean connectionBean = new ConnectionBean(connection);
|
||||
public ValidationResultBean update(String serverUrl, MLink link) {
|
||||
LinkBean linkBean = new LinkBean(link);
|
||||
|
||||
// Extract all form inputs including sensitive inputs
|
||||
JSONObject connectionJson = connectionBean.extract(false);
|
||||
JSONObject linkJson = linkBean.extract(false);
|
||||
|
||||
String response = super.put(serverUrl + RESOURCE + connection.getPersistenceId(), connectionJson.toJSONString());
|
||||
String response = super.put(serverUrl + RESOURCE + link.getPersistenceId(), linkJson.toJSONString());
|
||||
|
||||
ValidationResultBean validationBean = new ValidationResultBean();
|
||||
validationBean.restore((JSONObject) JSONValue.parse(response));
|
|
@ -34,8 +34,11 @@ import org.json.simple.JSONObject;
|
|||
import org.json.simple.JSONValue;
|
||||
|
||||
import java.util.Locale;
|
||||
|
||||
public class Request
|
||||
/**
|
||||
* Represents the sqoop REST resource requests
|
||||
*
|
||||
*/
|
||||
public class ResourceRequest
|
||||
{
|
||||
private static ServerExceptionFilter serverExceptionFilter;
|
||||
|
||||
|
@ -51,12 +54,12 @@ public class Request
|
|||
resource.addFilter(serverExceptionFilter);
|
||||
|
||||
return resource
|
||||
// Provide name of user executing request.
|
||||
.header(SqoopProtocolConstants.HEADER_SQOOP_USERNAME, System.getProperty("user.name"))
|
||||
// Sqoop is using JSON for data transfers
|
||||
.accept(MediaType.APPLICATION_JSON_TYPE)
|
||||
// Transfer client locale to return client specific data
|
||||
.acceptLanguage(Locale.getDefault());
|
||||
// Provide name of user executing request.
|
||||
.header(SqoopProtocolConstants.HEADER_SQOOP_USERNAME, System.getProperty("user.name"))
|
||||
// Sqoop is using JSON for data transfers
|
||||
.accept(MediaType.APPLICATION_JSON_TYPE)
|
||||
// Transfer client locale to return client specific data
|
||||
.acceptLanguage(Locale.getDefault());
|
||||
}
|
||||
|
||||
public String get(String url) {
|
|
@ -1,149 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.client.request;
|
||||
|
||||
import org.apache.sqoop.json.ConnectionBean;
|
||||
import org.apache.sqoop.json.ConnectorBean;
|
||||
import org.apache.sqoop.json.FrameworkBean;
|
||||
import org.apache.sqoop.json.JobBean;
|
||||
import org.apache.sqoop.json.SubmissionBean;
|
||||
import org.apache.sqoop.json.ValidationResultBean;
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MJob;
|
||||
|
||||
/**
|
||||
* Unified class for all request objects.
|
||||
*/
|
||||
public class SqoopRequests {
|
||||
|
||||
private String serverUrl;
|
||||
|
||||
private FrameworkRequest frameworkRequest;
|
||||
private ConnectorRequest connectorRequest;
|
||||
private ConnectionRequest connectionRequest;
|
||||
private JobRequest jobRequest;
|
||||
private SubmissionRequest submissionRequest;
|
||||
|
||||
public void setServerUrl(String serverUrl) {
|
||||
this.serverUrl = serverUrl;
|
||||
}
|
||||
|
||||
public FrameworkRequest getFrameworkRequest() {
|
||||
if (frameworkRequest == null) {
|
||||
frameworkRequest = new FrameworkRequest();
|
||||
}
|
||||
|
||||
return frameworkRequest;
|
||||
}
|
||||
|
||||
public ConnectorRequest getConnectorRequest() {
|
||||
if (connectorRequest == null) {
|
||||
connectorRequest = new ConnectorRequest();
|
||||
}
|
||||
|
||||
return connectorRequest;
|
||||
}
|
||||
|
||||
public ConnectionRequest getConnectionRequest() {
|
||||
if (connectionRequest == null) {
|
||||
connectionRequest = new ConnectionRequest();
|
||||
}
|
||||
|
||||
return connectionRequest;
|
||||
}
|
||||
|
||||
public JobRequest getJobRequest() {
|
||||
if (jobRequest == null) {
|
||||
jobRequest = new JobRequest();
|
||||
}
|
||||
|
||||
return jobRequest;
|
||||
}
|
||||
|
||||
public SubmissionRequest getSubmissionRequest() {
|
||||
if (submissionRequest == null) {
|
||||
submissionRequest = new SubmissionRequest();
|
||||
}
|
||||
|
||||
return submissionRequest;
|
||||
}
|
||||
|
||||
public FrameworkBean readFramework() {
|
||||
return getFrameworkRequest().read(serverUrl);
|
||||
}
|
||||
|
||||
public ConnectorBean readConnector(Long cid) {
|
||||
return getConnectorRequest().read(serverUrl, cid);
|
||||
}
|
||||
|
||||
public ValidationResultBean createConnection(MConnection connection) {
|
||||
return getConnectionRequest().create(serverUrl, connection);
|
||||
}
|
||||
|
||||
public ConnectionBean readConnection(Long connectionId) {
|
||||
return getConnectionRequest().read(serverUrl, connectionId);
|
||||
}
|
||||
|
||||
public ValidationResultBean updateConnection(MConnection connection) {
|
||||
return getConnectionRequest().update(serverUrl, connection);
|
||||
}
|
||||
|
||||
public void enableConnection(Long xid, Boolean enabled) {
|
||||
getConnectionRequest().enable(serverUrl, xid, enabled);
|
||||
}
|
||||
|
||||
public void deleteConnection(Long xid) {
|
||||
getConnectionRequest().delete(serverUrl, xid);
|
||||
}
|
||||
|
||||
public ValidationResultBean createJob(MJob job) {
|
||||
return getJobRequest().create(serverUrl, job);
|
||||
}
|
||||
|
||||
public JobBean readJob(Long jobId) {
|
||||
return getJobRequest().read(serverUrl, jobId);
|
||||
}
|
||||
|
||||
public ValidationResultBean updateJob(MJob job) {
|
||||
return getJobRequest().update(serverUrl, job);
|
||||
}
|
||||
|
||||
public void enableJob(Long jid, Boolean enabled) {
|
||||
getJobRequest().enable(serverUrl, jid, enabled);
|
||||
}
|
||||
|
||||
public void deleteJob(Long jid) {
|
||||
getJobRequest().delete(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean readHistory(Long jid) {
|
||||
return getSubmissionRequest().readHistory(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean readSubmission(Long jid) {
|
||||
return getSubmissionRequest().read(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean createSubmission(Long jid) {
|
||||
return getSubmissionRequest().create(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean deleteSubmission(Long jid) {
|
||||
return getSubmissionRequest().delete(serverUrl, jid);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,149 @@
|
|||
/**
|
||||
* 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.client.request;
|
||||
|
||||
import org.apache.sqoop.json.LinkBean;
|
||||
import org.apache.sqoop.json.ConnectorBean;
|
||||
import org.apache.sqoop.json.DriverConfigBean;
|
||||
import org.apache.sqoop.json.JobBean;
|
||||
import org.apache.sqoop.json.SubmissionBean;
|
||||
import org.apache.sqoop.json.ValidationResultBean;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.apache.sqoop.model.MJob;
|
||||
|
||||
/**
|
||||
* Unified class for all request objects.
|
||||
*/
|
||||
public class SqoopResourceRequests {
|
||||
|
||||
private String serverUrl;
|
||||
|
||||
private DriverConfigResourceRequest driverConfigRequest;
|
||||
private ConnectorResourceRequest connectorRequest;
|
||||
private LinkResourceRequest linkRequest;
|
||||
private JobResourceRequest jobRequest;
|
||||
private SubmissionResourceRequest submissionRequest;
|
||||
|
||||
public void setServerUrl(String serverUrl) {
|
||||
this.serverUrl = serverUrl;
|
||||
}
|
||||
|
||||
public DriverConfigResourceRequest getDriverConfigResourceRequest() {
|
||||
if (driverConfigRequest == null) {
|
||||
driverConfigRequest = new DriverConfigResourceRequest();
|
||||
}
|
||||
|
||||
return driverConfigRequest;
|
||||
}
|
||||
|
||||
public ConnectorResourceRequest getConnectorResourceRequest() {
|
||||
if (connectorRequest == null) {
|
||||
connectorRequest = new ConnectorResourceRequest();
|
||||
}
|
||||
|
||||
return connectorRequest;
|
||||
}
|
||||
|
||||
public LinkResourceRequest getLinkResourceRequest() {
|
||||
if (linkRequest == null) {
|
||||
linkRequest = new LinkResourceRequest();
|
||||
}
|
||||
|
||||
return linkRequest;
|
||||
}
|
||||
|
||||
public JobResourceRequest getJobResourceRequest() {
|
||||
if (jobRequest == null) {
|
||||
jobRequest = new JobResourceRequest();
|
||||
}
|
||||
|
||||
return jobRequest;
|
||||
}
|
||||
|
||||
public SubmissionResourceRequest getSubmissionResourceRequest() {
|
||||
if (submissionRequest == null) {
|
||||
submissionRequest = new SubmissionResourceRequest();
|
||||
}
|
||||
|
||||
return submissionRequest;
|
||||
}
|
||||
|
||||
public DriverConfigBean readDriverConfig() {
|
||||
return getDriverConfigResourceRequest().read(serverUrl);
|
||||
}
|
||||
|
||||
public ConnectorBean readConnector(Long cid) {
|
||||
return getConnectorResourceRequest().read(serverUrl, cid);
|
||||
}
|
||||
|
||||
public ValidationResultBean saveLink(MLink link) {
|
||||
return getLinkResourceRequest().create(serverUrl, link);
|
||||
}
|
||||
|
||||
public LinkBean readLink(Long linkId) {
|
||||
return getLinkResourceRequest().read(serverUrl, linkId);
|
||||
}
|
||||
|
||||
public ValidationResultBean updateLink(MLink link) {
|
||||
return getLinkResourceRequest().update(serverUrl, link);
|
||||
}
|
||||
|
||||
public void enableLink(Long lid, Boolean enabled) {
|
||||
getLinkResourceRequest().enable(serverUrl, lid, enabled);
|
||||
}
|
||||
|
||||
public void deleteLink(Long lid) {
|
||||
getLinkResourceRequest().delete(serverUrl, lid);
|
||||
}
|
||||
|
||||
public ValidationResultBean saveJob(MJob job) {
|
||||
return getJobResourceRequest().create(serverUrl, job);
|
||||
}
|
||||
|
||||
public JobBean readJob(Long jobId) {
|
||||
return getJobResourceRequest().read(serverUrl, jobId);
|
||||
}
|
||||
|
||||
public ValidationResultBean updateJob(MJob job) {
|
||||
return getJobResourceRequest().update(serverUrl, job);
|
||||
}
|
||||
|
||||
public void enableJob(Long jid, Boolean enabled) {
|
||||
getJobResourceRequest().enable(serverUrl, jid, enabled);
|
||||
}
|
||||
|
||||
public void deleteJob(Long jid) {
|
||||
getJobResourceRequest().delete(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean readHistory(Long jid) {
|
||||
return getSubmissionResourceRequest().readHistory(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean readSubmission(Long jid) {
|
||||
return getSubmissionResourceRequest().read(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean createSubmission(Long jid) {
|
||||
return getSubmissionResourceRequest().create(serverUrl, jid);
|
||||
}
|
||||
|
||||
public SubmissionBean deleteSubmission(Long jid) {
|
||||
return getSubmissionResourceRequest().delete(serverUrl, jid);
|
||||
}
|
||||
}
|
|
@ -25,7 +25,7 @@ import org.json.simple.JSONValue;
|
|||
* Provide CRD semantics over RESTfull HTTP API for submissions. Please note
|
||||
* that "update" is not supported as client can't update submission status.
|
||||
*/
|
||||
public class SubmissionRequest extends Request {
|
||||
public class SubmissionResourceRequest extends ResourceRequest {
|
||||
|
||||
public static final String RESOURCE = "v1/submission/";
|
||||
|
|
@ -21,9 +21,9 @@ import org.apache.sqoop.json.VersionBean;
|
|||
import org.json.simple.JSONObject;
|
||||
import org.json.simple.JSONValue;
|
||||
|
||||
public class VersionRequest extends Request
|
||||
public class VersionResourceRequest extends ResourceRequest
|
||||
{
|
||||
public VersionBean doGet(String serverUrl) {
|
||||
public VersionBean read(String serverUrl) {
|
||||
String response = super.get(serverUrl + "version");
|
||||
JSONObject jsonObject = (JSONObject)JSONValue.parse(response);
|
||||
|
|
@ -17,13 +17,13 @@
|
|||
*/
|
||||
package org.apache.sqoop.client;
|
||||
|
||||
import org.apache.sqoop.client.request.SqoopRequests;
|
||||
import org.apache.sqoop.client.request.SqoopResourceRequests;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.json.ConnectorBean;
|
||||
import org.apache.sqoop.json.FrameworkBean;
|
||||
import org.apache.sqoop.json.DriverConfigBean;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MConnector;
|
||||
import org.apache.sqoop.model.MFramework;
|
||||
import org.apache.sqoop.model.MDriverConfig;
|
||||
import org.apache.sqoop.model.MJobForms;
|
||||
import org.apache.sqoop.utils.MapResourceBundle;
|
||||
import org.junit.Before;
|
||||
|
@ -32,7 +32,6 @@ import org.junit.Test;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.ResourceBundle;
|
||||
|
@ -43,14 +42,14 @@ import static org.mockito.Mockito.*;
|
|||
|
||||
public class TestSqoopClient {
|
||||
|
||||
SqoopRequests requests;
|
||||
SqoopResourceRequests resourceRequests;
|
||||
SqoopClient client;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
requests = mock(SqoopRequests.class);
|
||||
resourceRequests = mock(SqoopResourceRequests.class);
|
||||
client = new SqoopClient("my-cool-server");
|
||||
client.setSqoopRequests(requests);
|
||||
client.setSqoopRequests(resourceRequests);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -59,26 +58,26 @@ public class TestSqoopClient {
|
|||
*/
|
||||
@Test
|
||||
public void testGetConnector() {
|
||||
when(requests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
|
||||
when(resourceRequests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
|
||||
MConnector connector = client.getConnector(1);
|
||||
assertEquals(1, connector.getPersistenceId());
|
||||
|
||||
client.getResourceBundle(1L);
|
||||
client.getConnectorConfigResourceBundle(1L);
|
||||
|
||||
verify(requests, times(1)).readConnector(1L);
|
||||
verify(resourceRequests, times(1)).readConnector(1L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetConnectorByString() {
|
||||
when(requests.readConnector(null)).thenReturn(connectorBean(connector(1)));
|
||||
when(resourceRequests.readConnector(null)).thenReturn(connectorBean(connector(1)));
|
||||
MConnector connector = client.getConnector("A1");
|
||||
assertEquals(1, connector.getPersistenceId());
|
||||
assertEquals("A1", connector.getUniqueName());
|
||||
|
||||
client.getResourceBundle(1L);
|
||||
client.getConnectorConfigResourceBundle(1L);
|
||||
|
||||
verify(requests, times(0)).readConnector(1L);
|
||||
verify(requests, times(1)).readConnector(null);
|
||||
verify(resourceRequests, times(0)).readConnector(1L);
|
||||
verify(resourceRequests, times(1)).readConnector(null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -87,41 +86,41 @@ public class TestSqoopClient {
|
|||
*/
|
||||
@Test
|
||||
public void testGetConnectorBundle() {
|
||||
when(requests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
|
||||
client.getResourceBundle(1L);
|
||||
when(resourceRequests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
|
||||
client.getConnectorConfigResourceBundle(1L);
|
||||
|
||||
MConnector connector = client.getConnector(1);
|
||||
assertEquals(1, connector.getPersistenceId());
|
||||
|
||||
verify(requests, times(1)).readConnector(1L);
|
||||
verify(resourceRequests, times(1)).readConnector(1L);
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve framework information, request to framework bundle should not
|
||||
* Retrieve driverConfig information, request to driverConfig bundle should not
|
||||
* require additional HTTP request.
|
||||
*/
|
||||
@Test
|
||||
public void testGetFramework() {
|
||||
when(requests.readFramework()).thenReturn(frameworkBean(framework()));
|
||||
public void testGetDriverConfig() {
|
||||
when(resourceRequests.readDriverConfig()).thenReturn(driverConfigBean(driverConfig()));
|
||||
|
||||
client.getFramework();
|
||||
client.getFrameworkResourceBundle();
|
||||
client.getDriverConfig();
|
||||
client.getDriverConfigBundle();
|
||||
|
||||
verify(requests, times(1)).readFramework();
|
||||
verify(resourceRequests, times(1)).readDriverConfig();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve framework bundle, request to framework metadata should not
|
||||
* Retrieve driverConfig bundle, request to driverConfig metadata should not
|
||||
* require additional HTTP request.
|
||||
*/
|
||||
@Test
|
||||
public void testGetFrameworkBundle() {
|
||||
when(requests.readFramework()).thenReturn(frameworkBean(framework()));
|
||||
public void testGetDriverConfigBundle() {
|
||||
when(resourceRequests.readDriverConfig()).thenReturn(driverConfigBean(driverConfig()));
|
||||
|
||||
client.getFrameworkResourceBundle();
|
||||
client.getFramework();
|
||||
client.getDriverConfigBundle();
|
||||
client.getDriverConfig();
|
||||
|
||||
verify(requests, times(1)).readFramework();
|
||||
verify(resourceRequests, times(1)).readDriverConfig();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -132,16 +131,16 @@ public class TestSqoopClient {
|
|||
public void testGetConnectors() {
|
||||
MConnector connector;
|
||||
|
||||
when(requests.readConnector(null)).thenReturn(connectorBean(connector(1), connector(2)));
|
||||
when(resourceRequests.readConnector(null)).thenReturn(connectorBean(connector(1), connector(2)));
|
||||
Collection<MConnector> connectors = client.getConnectors();
|
||||
assertEquals(2, connectors.size());
|
||||
|
||||
client.getResourceBundle(1);
|
||||
client.getConnectorConfigResourceBundle(1);
|
||||
connector = client.getConnector(1);
|
||||
assertEquals(1, connector.getPersistenceId());
|
||||
|
||||
connector = client.getConnector(2);
|
||||
client.getResourceBundle(2);
|
||||
client.getConnectorConfigResourceBundle(2);
|
||||
assertEquals(2, connector.getPersistenceId());
|
||||
|
||||
connectors = client.getConnectors();
|
||||
|
@ -158,8 +157,8 @@ public class TestSqoopClient {
|
|||
connector = client.getConnector("A3");
|
||||
assertNull(connector);
|
||||
|
||||
verify(requests, times(1)).readConnector(null);
|
||||
verifyNoMoreInteractions(requests);
|
||||
verify(resourceRequests, times(1)).readConnector(null);
|
||||
verifyNoMoreInteractions(resourceRequests);
|
||||
}
|
||||
|
||||
|
||||
|
@ -170,32 +169,32 @@ public class TestSqoopClient {
|
|||
@Test
|
||||
public void testGetConnectorOneByOne() {
|
||||
ConnectorBean bean = connectorBean(connector(1), connector(2));
|
||||
when(requests.readConnector(null)).thenReturn(bean);
|
||||
when(requests.readConnector(1L)).thenReturn(bean);
|
||||
when(requests.readConnector(2L)).thenReturn(bean);
|
||||
when(resourceRequests.readConnector(null)).thenReturn(bean);
|
||||
when(resourceRequests.readConnector(1L)).thenReturn(bean);
|
||||
when(resourceRequests.readConnector(2L)).thenReturn(bean);
|
||||
|
||||
client.getResourceBundle(1);
|
||||
client.getConnectorConfigResourceBundle(1);
|
||||
client.getConnector(1);
|
||||
|
||||
client.getConnector(2);
|
||||
client.getResourceBundle(2);
|
||||
client.getConnectorConfigResourceBundle(2);
|
||||
|
||||
Collection<MConnector> connectors = client.getConnectors();
|
||||
assertEquals(2, connectors.size());
|
||||
|
||||
verify(requests, times(1)).readConnector(null);
|
||||
verify(requests, times(1)).readConnector(1L);
|
||||
verify(requests, times(1)).readConnector(2L);
|
||||
verifyNoMoreInteractions(requests);
|
||||
verify(resourceRequests, times(1)).readConnector(null);
|
||||
verify(resourceRequests, times(1)).readConnector(1L);
|
||||
verify(resourceRequests, times(1)).readConnector(2L);
|
||||
verifyNoMoreInteractions(resourceRequests);
|
||||
}
|
||||
|
||||
/**
|
||||
* Connection for non-existing connector can't be created.
|
||||
* Link for non-existing connector can't be created.
|
||||
*/
|
||||
@Test(expected = SqoopException.class)
|
||||
public void testNewConnection() {
|
||||
when(requests.readConnector(null)).thenReturn(connectorBean(connector(1)));
|
||||
client.newConnection("non existing connector");
|
||||
public void testCreateLink() {
|
||||
when(resourceRequests.readConnector(null)).thenReturn(connectorBean(connector(1)));
|
||||
client.createLink("non existing connector");
|
||||
}
|
||||
|
||||
private ConnectorBean connectorBean(MConnector...connectors) {
|
||||
|
@ -208,8 +207,8 @@ public class TestSqoopClient {
|
|||
}
|
||||
return new ConnectorBean(connectorList, bundles);
|
||||
}
|
||||
private FrameworkBean frameworkBean(MFramework framework) {
|
||||
return new FrameworkBean(framework, new MapResourceBundle(null));
|
||||
private DriverConfigBean driverConfigBean(MDriverConfig driverConfig) {
|
||||
return new DriverConfigBean(driverConfig, new MapResourceBundle(null));
|
||||
}
|
||||
|
||||
private MConnector connector(long id) {
|
||||
|
@ -219,10 +218,10 @@ public class TestSqoopClient {
|
|||
return connector;
|
||||
}
|
||||
|
||||
private MFramework framework() {
|
||||
MFramework framework = new MFramework(new MConnectionForms(null),
|
||||
private MDriverConfig driverConfig() {
|
||||
MDriverConfig driverConfig = new MDriverConfig(new MConnectionForms(null),
|
||||
new MJobForms(null), "1");
|
||||
framework.setPersistenceId(1);
|
||||
return framework;
|
||||
driverConfig.setPersistenceId(1);
|
||||
return driverConfig;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.sqoop.common;
|
|||
|
||||
|
||||
/**
|
||||
* Base exception for Sqoop framework. This exception requires the specification
|
||||
* Base exception for Sqoop driver. 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.
|
||||
|
|
|
@ -18,20 +18,20 @@
|
|||
package org.apache.sqoop.etl.io;
|
||||
|
||||
/**
|
||||
* An intermediate layer for passing data from the execution framework
|
||||
* to the ETL framework.
|
||||
* An intermediate layer for passing data from the execution engine
|
||||
* to the ETL engine.
|
||||
*/
|
||||
public abstract class DataReader {
|
||||
|
||||
/**
|
||||
* Read data from the execution framework as an object array.
|
||||
* Read data from the execution engine as an object array.
|
||||
* @return - array of objects with each column represented as an object
|
||||
* @throws Exception
|
||||
*/
|
||||
public abstract Object[] readArrayRecord() throws Exception;
|
||||
|
||||
/**
|
||||
* Read data from execution framework as text - as a CSV record.
|
||||
* Read data from execution engine as text - as a CSV record.
|
||||
* public abstract Object readContent(int type) throws Exception;
|
||||
* @return - CSV formatted data.
|
||||
* @throws Exception
|
||||
|
@ -39,7 +39,7 @@ public abstract class DataReader {
|
|||
public abstract String readTextRecord() throws Exception;
|
||||
|
||||
/**
|
||||
* Read data from execution framework as a native format.
|
||||
* Read data from execution engine as a native format.
|
||||
* @return - the content in the native format of the intermediate data
|
||||
* format being used.
|
||||
* @throws Exception
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.sqoop.schema.Schema;
|
|||
*
|
||||
* This class is wrapping information if the run was successful or not.
|
||||
*/
|
||||
public class DestroyerContext extends ActorContext {
|
||||
public class DestroyerContext extends TransferableContext {
|
||||
|
||||
private boolean success;
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.sqoop.schema.Schema;
|
|||
*
|
||||
* This class is wrapping writer object.
|
||||
*/
|
||||
public class ExtractorContext extends ActorContext {
|
||||
public class ExtractorContext extends TransferableContext {
|
||||
|
||||
private DataWriter writer;
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.sqoop.common.MutableContext;
|
|||
*
|
||||
* This class is returning mutable context instead of immutable.
|
||||
*/
|
||||
public class InitializerContext extends ActorContext {
|
||||
public class InitializerContext extends TransferableContext {
|
||||
|
||||
public InitializerContext(MutableContext context) {
|
||||
super(context);
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.sqoop.schema.Schema;
|
|||
*
|
||||
* This class is also wrapping reader object.
|
||||
*/
|
||||
public class LoaderContext extends ActorContext {
|
||||
public class LoaderContext extends TransferableContext {
|
||||
|
||||
private DataReader reader;
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.sqoop.schema.Schema;
|
|||
*
|
||||
* This class is also wrapping number of maximal allowed partitions.
|
||||
*/
|
||||
public class PartitionerContext extends ActorContext {
|
||||
public class PartitionerContext extends TransferableContext {
|
||||
|
||||
private long maxPartitions;
|
||||
|
||||
|
|
|
@ -20,14 +20,13 @@ package org.apache.sqoop.job.etl;
|
|||
import org.apache.sqoop.common.ImmutableContext;
|
||||
|
||||
/**
|
||||
* Basic context class for each actor containing only the connector/framework
|
||||
* context object.
|
||||
* Base context class for the {@link Transferable} components
|
||||
*/
|
||||
public abstract class ActorContext {
|
||||
public abstract class TransferableContext {
|
||||
|
||||
ImmutableContext context;
|
||||
|
||||
public ActorContext(ImmutableContext context) {
|
||||
public TransferableContext(ImmutableContext context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
|
@ -17,6 +17,19 @@
|
|||
*/
|
||||
package org.apache.sqoop.json;
|
||||
|
||||
import static org.apache.sqoop.json.util.FormSerialization.ALL;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.CLASS;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.CON_FORMS;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.ID;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.JOB_FORMS;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.NAME;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.VERSION;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.extractForms;
|
||||
import static org.apache.sqoop.json.util.FormSerialization.restoreForms;
|
||||
import static org.apache.sqoop.json.util.ResourceBundleSerialization.CONNECTOR_CONFIGS;
|
||||
import static org.apache.sqoop.json.util.ResourceBundleSerialization.extractResourceBundle;
|
||||
import static org.apache.sqoop.json.util.ResourceBundleSerialization.restoreResourceBundle;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -26,15 +39,12 @@ import java.util.Set;
|
|||
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MJobForms;
|
||||
import org.apache.sqoop.model.MConnector;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.apache.sqoop.model.MJobForms;
|
||||
import org.json.simple.JSONArray;
|
||||
import org.json.simple.JSONObject;
|
||||
|
||||
import static org.apache.sqoop.json.util.FormSerialization.*;
|
||||
import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
|
||||
|
||||
public class ConnectorBean implements JsonBean {
|
||||
|
||||
private List<MConnector> connectors;
|
||||
|
@ -93,8 +103,7 @@ public class ConnectorBean implements JsonBean {
|
|||
jsonBundles.put(entry.getKey().toString(),
|
||||
extractResourceBundle(entry.getValue()));
|
||||
}
|
||||
|
||||
all.put(CONNECTOR_RESOURCES, jsonBundles);
|
||||
all.put(CONNECTOR_CONFIGS, jsonBundles);
|
||||
}
|
||||
|
||||
return all;
|
||||
|
@ -116,28 +125,24 @@ public class ConnectorBean implements JsonBean {
|
|||
String version = (String) object.get(VERSION);
|
||||
|
||||
List<MForm> connForms = restoreForms((JSONArray) object.get(CON_FORMS));
|
||||
|
||||
JSONObject jobJson = (JSONObject) object.get(JOB_FORMS);
|
||||
JSONArray fromJobJson = (JSONArray)jobJson.get(Direction.FROM.name());
|
||||
JSONArray toJobJson = (JSONArray)jobJson.get(Direction.TO.name());
|
||||
List<MForm> fromJobForms =
|
||||
restoreForms(fromJobJson);
|
||||
List<MForm> toJobForms =
|
||||
restoreForms(toJobJson);
|
||||
List<MForm> fromJobForms = restoreForms(fromJobJson);
|
||||
List<MForm> toJobForms = restoreForms(toJobJson);
|
||||
MJobForms fromJob = new MJobForms(fromJobForms);
|
||||
MJobForms toJob = new MJobForms(toJobForms);
|
||||
MConnectionForms connection = new MConnectionForms(connForms);
|
||||
|
||||
MConnector connector = new MConnector(uniqueName, className, version, connection, fromJob, toJob);
|
||||
MConnector connector = new MConnector(uniqueName, className, version, connection, fromJob,
|
||||
toJob);
|
||||
connector.setPersistenceId(connectorId);
|
||||
|
||||
connectors.add(connector);
|
||||
}
|
||||
|
||||
if(jsonObject.containsKey(CONNECTOR_RESOURCES)) {
|
||||
if(jsonObject.containsKey(CONNECTOR_CONFIGS)) {
|
||||
bundles = new HashMap<Long, ResourceBundle>();
|
||||
|
||||
JSONObject jsonBundles = (JSONObject) jsonObject.get(CONNECTOR_RESOURCES);
|
||||
JSONObject jsonBundles = (JSONObject) jsonObject.get(CONNECTOR_CONFIGS);
|
||||
Set<Map.Entry<String, JSONObject>> entrySet = jsonBundles.entrySet();
|
||||
for (Map.Entry<String, JSONObject> entry : entrySet) {
|
||||
bundles.put(Long.parseLong(entry.getKey()),
|
||||
|
|
|
@ -18,45 +18,36 @@
|
|||
package org.apache.sqoop.json;
|
||||
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MConnector;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.apache.sqoop.model.MFramework;
|
||||
import org.apache.sqoop.model.MJob;
|
||||
import org.apache.sqoop.model.MDriverConfig;
|
||||
import org.apache.sqoop.model.MJobForms;
|
||||
import org.json.simple.JSONArray;
|
||||
import org.json.simple.JSONObject;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.ResourceBundle;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.sqoop.json.util.FormSerialization.*;
|
||||
import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class FrameworkBean implements JsonBean {
|
||||
public class DriverConfigBean implements JsonBean {
|
||||
|
||||
|
||||
private MFramework framework;
|
||||
private MDriverConfig driverConfig;
|
||||
|
||||
private ResourceBundle bundle;
|
||||
|
||||
// for "extract"
|
||||
public FrameworkBean(MFramework framework, ResourceBundle bundle) {
|
||||
this.framework = framework;
|
||||
public DriverConfigBean(MDriverConfig driverConfig, ResourceBundle bundle) {
|
||||
this.driverConfig = driverConfig;
|
||||
this.bundle = bundle;
|
||||
}
|
||||
|
||||
// for "restore"
|
||||
public FrameworkBean() {
|
||||
public DriverConfigBean() {
|
||||
}
|
||||
|
||||
public MFramework getFramework() {
|
||||
return framework;
|
||||
public MDriverConfig getDriverConfig() {
|
||||
return driverConfig;
|
||||
}
|
||||
|
||||
public ResourceBundle getResourceBundle() {
|
||||
|
@ -66,37 +57,36 @@ public class FrameworkBean implements JsonBean {
|
|||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public JSONObject extract(boolean skipSensitive) {
|
||||
// @TODO(Abe): Add From/To connection forms.
|
||||
// TODO(Abe): Add From/To connection forms.
|
||||
JSONArray conForms =
|
||||
extractForms(framework.getConnectionForms().getForms(), skipSensitive);
|
||||
JSONArray jobForms = extractForms(framework.getJobForms().getForms(), skipSensitive);
|
||||
extractForms(driverConfig.getConnectionForms().getForms(), skipSensitive);
|
||||
JSONArray jobForms = extractForms(driverConfig.getJobForms().getForms(), skipSensitive);
|
||||
|
||||
JSONObject result = new JSONObject();
|
||||
result.put(ID, framework.getPersistenceId());
|
||||
result.put(FRAMEWORK_VERSION, framework.getVersion());
|
||||
result.put(ID, driverConfig.getPersistenceId());
|
||||
result.put(DRIVER_VERSION, driverConfig.getVersion());
|
||||
result.put(CON_FORMS, conForms);
|
||||
result.put(JOB_FORMS, jobForms);
|
||||
result.put(RESOURCES, extractResourceBundle(bundle));
|
||||
result.put(CONFIGS, extractResourceBundle(bundle));
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public void restore(JSONObject jsonObject) {
|
||||
long id = (Long) jsonObject.get(ID);
|
||||
String frameworkVersion = (String) jsonObject.get(FRAMEWORK_VERSION);
|
||||
String driverVersion = (String) jsonObject.get(DRIVER_VERSION);
|
||||
|
||||
List<MForm> connForms = restoreForms((JSONArray) jsonObject.get(CON_FORMS));
|
||||
List<MForm> jobForms = restoreForms((JSONArray) jsonObject.get(JOB_FORMS));
|
||||
|
||||
// @TODO(Abe): Get From/To connection forms.
|
||||
framework = new MFramework(
|
||||
// TODO(Abe): Get From/To connection forms.
|
||||
driverConfig = new MDriverConfig(
|
||||
new MConnectionForms(connForms),
|
||||
new MJobForms(jobForms),
|
||||
frameworkVersion);
|
||||
framework.setPersistenceId(id);
|
||||
driverVersion);
|
||||
driverConfig.setPersistenceId(id);
|
||||
|
||||
bundle = restoreResourceBundle((JSONObject) jsonObject.get(RESOURCES));
|
||||
bundle = restoreResourceBundle((JSONObject) jsonObject.get(CONFIGS));
|
||||
}
|
||||
|
||||
}
|
|
@ -43,20 +43,20 @@ public class JobBean implements JsonBean {
|
|||
private static final String ALL = "all";
|
||||
private static final String ID = "id";
|
||||
private static final String NAME = "name";
|
||||
private static final String FROM_CONNECTION_ID = "from-connection-id";
|
||||
private static final String TO_CONNECTION_ID = "to-connection-id";
|
||||
private static final String FROM_LINK_ID = "from-link-id";
|
||||
private static final String TO_LINK_ID = "to-link-id";
|
||||
private static final String FROM_CONNECTOR_ID = "from-connector-id";
|
||||
private static final String TO_CONNECTOR_ID = "to-connector-id";
|
||||
private static final String FROM_CONNECTOR_PART = "from-connector";
|
||||
private static final String TO_CONNECTOR_PART = "to-connector";
|
||||
private static final String FRAMEWORK_PART = "framework";
|
||||
|
||||
// Compulsory
|
||||
// Required
|
||||
private List<MJob> jobs;
|
||||
|
||||
// Optional
|
||||
private Map<Long, ResourceBundle> connectorBundles;
|
||||
private ResourceBundle frameworkBundle;
|
||||
private Map<Long, ResourceBundle> connectorConfigBundles;
|
||||
private ResourceBundle driverConfigBundle;
|
||||
|
||||
// For "extract"
|
||||
public JobBean(MJob job) {
|
||||
|
@ -72,31 +72,31 @@ public class JobBean implements JsonBean {
|
|||
|
||||
// For "restore"
|
||||
public JobBean() {
|
||||
connectorBundles = new HashMap<Long, ResourceBundle>();
|
||||
connectorConfigBundles = new HashMap<Long, ResourceBundle>();
|
||||
}
|
||||
|
||||
public void setFrameworkBundle(ResourceBundle frameworkBundle) {
|
||||
this.frameworkBundle = frameworkBundle;
|
||||
public void setDriverConfigBundle(ResourceBundle driverConfigBundle) {
|
||||
this.driverConfigBundle = driverConfigBundle;
|
||||
}
|
||||
|
||||
public void addConnectorBundle(Long id, ResourceBundle connectorBundle) {
|
||||
connectorBundles.put(id, connectorBundle);
|
||||
public void addConnectorConfigBundle(Long id, ResourceBundle connectorConfigBundle) {
|
||||
connectorConfigBundles.put(id, connectorConfigBundle);
|
||||
}
|
||||
|
||||
public boolean hasConnectorBundle(Long id) {
|
||||
return connectorBundles.containsKey(id);
|
||||
public boolean hasConnectorConfigBundle(Long id) {
|
||||
return connectorConfigBundles.containsKey(id);
|
||||
}
|
||||
|
||||
public List<MJob> getJobs() {
|
||||
return jobs;
|
||||
}
|
||||
|
||||
public ResourceBundle getConnectorBundle(Long id) {
|
||||
return connectorBundles.get(id);
|
||||
public ResourceBundle getConnectorConfigBundle(Long id) {
|
||||
return connectorConfigBundles.get(id);
|
||||
}
|
||||
|
||||
public ResourceBundle getFrameworkBundle() {
|
||||
return frameworkBundle;
|
||||
public ResourceBundle getDriverConfigBundle() {
|
||||
return driverConfigBundle;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -114,8 +114,8 @@ public class JobBean implements JsonBean {
|
|||
object.put(CREATION_DATE, job.getCreationDate().getTime());
|
||||
object.put(UPDATE_USER, job.getLastUpdateUser());
|
||||
object.put(UPDATE_DATE, job.getLastUpdateDate().getTime());
|
||||
object.put(FROM_CONNECTION_ID, job.getConnectionId(Direction.FROM));
|
||||
object.put(TO_CONNECTION_ID, job.getConnectionId(Direction.TO));
|
||||
object.put(FROM_LINK_ID, job.getLinkId(Direction.FROM));
|
||||
object.put(TO_LINK_ID, job.getLinkId(Direction.TO));
|
||||
object.put(FROM_CONNECTOR_ID, job.getConnectorId(Direction.FROM));
|
||||
object.put(TO_CONNECTOR_ID, job.getConnectorId(Direction.TO));
|
||||
object.put(FROM_CONNECTOR_PART,
|
||||
|
@ -131,18 +131,17 @@ public class JobBean implements JsonBean {
|
|||
JSONObject all = new JSONObject();
|
||||
all.put(ALL, array);
|
||||
|
||||
if(!connectorBundles.isEmpty()) {
|
||||
if(!connectorConfigBundles.isEmpty()) {
|
||||
JSONObject bundles = new JSONObject();
|
||||
|
||||
for(Map.Entry<Long, ResourceBundle> entry : connectorBundles.entrySet()) {
|
||||
for(Map.Entry<Long, ResourceBundle> entry : connectorConfigBundles.entrySet()) {
|
||||
bundles.put(entry.getKey().toString(),
|
||||
extractResourceBundle(entry.getValue()));
|
||||
}
|
||||
|
||||
all.put(CONNECTOR_RESOURCES, bundles);
|
||||
all.put(CONNECTOR_CONFIGS, bundles);
|
||||
}
|
||||
if(frameworkBundle != null) {
|
||||
all.put(FRAMEWORK_RESOURCES,extractResourceBundle(frameworkBundle));
|
||||
if(driverConfigBundle != null) {
|
||||
all.put(DRIVER_CONFIGS,extractResourceBundle(driverConfigBundle));
|
||||
}
|
||||
return all;
|
||||
}
|
||||
|
@ -159,8 +158,8 @@ public class JobBean implements JsonBean {
|
|||
|
||||
long fromConnectorId = (Long) object.get(FROM_CONNECTOR_ID);
|
||||
long toConnectorId = (Long) object.get(TO_CONNECTOR_ID);
|
||||
long fromConnectionId = (Long) object.get(FROM_CONNECTION_ID);
|
||||
long toConnectionId = (Long) object.get(TO_CONNECTION_ID);
|
||||
long fromConnectionId = (Long) object.get(FROM_LINK_ID);
|
||||
long toConnectionId = (Long) object.get(TO_LINK_ID);
|
||||
JSONArray fromConnectorPart = (JSONArray) object.get(FROM_CONNECTOR_PART);
|
||||
JSONArray toConnectorPart = (JSONArray) object.get(TO_CONNECTOR_PART);
|
||||
JSONArray frameworkPart = (JSONArray) object.get(FRAMEWORK_PART);
|
||||
|
@ -190,17 +189,17 @@ public class JobBean implements JsonBean {
|
|||
jobs.add(job);
|
||||
}
|
||||
|
||||
if(jsonObject.containsKey(CONNECTOR_RESOURCES)) {
|
||||
JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_RESOURCES);
|
||||
if(jsonObject.containsKey(CONNECTOR_CONFIGS)) {
|
||||
JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_CONFIGS);
|
||||
Set<Map.Entry<String, JSONObject>> entrySet = bundles.entrySet();
|
||||
for (Map.Entry<String, JSONObject> entry : entrySet) {
|
||||
connectorBundles.put(Long.parseLong(entry.getKey()),
|
||||
connectorConfigBundles.put(Long.parseLong(entry.getKey()),
|
||||
restoreResourceBundle(entry.getValue()));
|
||||
}
|
||||
}
|
||||
if(jsonObject.containsKey(FRAMEWORK_RESOURCES)) {
|
||||
frameworkBundle = restoreResourceBundle(
|
||||
(JSONObject) jsonObject.get(FRAMEWORK_RESOURCES));
|
||||
if(jsonObject.containsKey(DRIVER_CONFIGS)) {
|
||||
driverConfigBundle = restoreResourceBundle(
|
||||
(JSONObject) jsonObject.get(DRIVER_CONFIGS));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
package org.apache.sqoop.json;
|
||||
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.json.simple.JSONArray;
|
||||
|
@ -35,63 +35,63 @@ import static org.apache.sqoop.json.util.FormSerialization.*;
|
|||
import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
|
||||
|
||||
/**
|
||||
* Connection representation that is being send across the network between
|
||||
* Sqoop server and client. Server might optionally send resource bundles
|
||||
* associated with the connections to spare client of sending another HTTP
|
||||
* Link representation that is being send across the network between
|
||||
* Sqoop server and client. Server might optionally send configs
|
||||
* associated with the links to spare client of sending another HTTP
|
||||
* requests to obtain them.
|
||||
*/
|
||||
public class ConnectionBean implements JsonBean {
|
||||
public class LinkBean implements JsonBean {
|
||||
|
||||
private static final String CONNECTOR_ID = "connector-id";
|
||||
private static final String CONNECTOR_PART = "connector";
|
||||
private static final String FRAMEWORK_PART = "framework";
|
||||
|
||||
// Compulsory
|
||||
private List<MConnection> connections;
|
||||
// Required
|
||||
private List<MLink> links;
|
||||
|
||||
// Optional
|
||||
private Map<Long, ResourceBundle> connectorBundles;
|
||||
private ResourceBundle frameworkBundle;
|
||||
private Map<Long, ResourceBundle> connectorConfigBundles;
|
||||
private ResourceBundle driverConfigBundle;
|
||||
|
||||
// For "extract"
|
||||
public ConnectionBean(MConnection connection) {
|
||||
public LinkBean(MLink link) {
|
||||
this();
|
||||
this.connections = new ArrayList<MConnection>();
|
||||
this.connections.add(connection);
|
||||
this.links = new ArrayList<MLink>();
|
||||
this.links.add(link);
|
||||
}
|
||||
|
||||
public ConnectionBean(List<MConnection> connections) {
|
||||
public LinkBean(List<MLink> links) {
|
||||
this();
|
||||
this.connections = connections;
|
||||
this.links = links;
|
||||
}
|
||||
|
||||
// For "restore"
|
||||
public ConnectionBean() {
|
||||
connectorBundles = new HashMap<Long, ResourceBundle>();
|
||||
public LinkBean() {
|
||||
connectorConfigBundles = new HashMap<Long, ResourceBundle>();
|
||||
}
|
||||
|
||||
public void setFrameworkBundle(ResourceBundle frameworkBundle) {
|
||||
this.frameworkBundle = frameworkBundle;
|
||||
public void setDriverConfigBundle(ResourceBundle driverConfigBundle) {
|
||||
this.driverConfigBundle = driverConfigBundle;
|
||||
}
|
||||
|
||||
public void addConnectorBundle(Long id, ResourceBundle connectorBundle) {
|
||||
connectorBundles.put(id, connectorBundle);
|
||||
public void addConnectorConfigBundle(Long id, ResourceBundle connectorConfigBundle) {
|
||||
connectorConfigBundles.put(id, connectorConfigBundle);
|
||||
}
|
||||
|
||||
public boolean hasConnectorBundle(Long id) {
|
||||
return connectorBundles.containsKey(id);
|
||||
return connectorConfigBundles.containsKey(id);
|
||||
}
|
||||
|
||||
public List<MConnection> getConnections() {
|
||||
return connections;
|
||||
public List<MLink> getLinks() {
|
||||
return links;
|
||||
}
|
||||
|
||||
public ResourceBundle getConnectorBundle(Long id) {
|
||||
return connectorBundles.get(id);
|
||||
return connectorConfigBundles.get(id);
|
||||
}
|
||||
|
||||
public ResourceBundle getFrameworkBundle() {
|
||||
return frameworkBundle;
|
||||
return driverConfigBundle;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -99,21 +99,21 @@ public class ConnectionBean implements JsonBean {
|
|||
public JSONObject extract(boolean skipSensitive) {
|
||||
JSONArray array = new JSONArray();
|
||||
|
||||
for(MConnection connection : connections) {
|
||||
for(MLink link : links) {
|
||||
JSONObject object = new JSONObject();
|
||||
|
||||
object.put(ID, connection.getPersistenceId());
|
||||
object.put(NAME, connection.getName());
|
||||
object.put(ENABLED, connection.getEnabled());
|
||||
object.put(CREATION_USER, connection.getCreationUser());
|
||||
object.put(CREATION_DATE, connection.getCreationDate().getTime());
|
||||
object.put(UPDATE_USER, connection.getLastUpdateUser());
|
||||
object.put(UPDATE_DATE, connection.getLastUpdateDate().getTime());
|
||||
object.put(CONNECTOR_ID, connection.getConnectorId());
|
||||
object.put(ID, link.getPersistenceId());
|
||||
object.put(NAME, link.getName());
|
||||
object.put(ENABLED, link.getEnabled());
|
||||
object.put(CREATION_USER, link.getCreationUser());
|
||||
object.put(CREATION_DATE, link.getCreationDate().getTime());
|
||||
object.put(UPDATE_USER, link.getLastUpdateUser());
|
||||
object.put(UPDATE_DATE, link.getLastUpdateDate().getTime());
|
||||
object.put(CONNECTOR_ID, link.getConnectorId());
|
||||
object.put(CONNECTOR_PART,
|
||||
extractForms(connection.getConnectorPart().getForms(), skipSensitive));
|
||||
extractForms(link.getConnectorPart().getForms(), skipSensitive));
|
||||
object.put(FRAMEWORK_PART,
|
||||
extractForms(connection.getFrameworkPart().getForms(), skipSensitive));
|
||||
extractForms(link.getFrameworkPart().getForms(), skipSensitive));
|
||||
|
||||
array.add(object);
|
||||
}
|
||||
|
@ -121,18 +121,17 @@ public class ConnectionBean implements JsonBean {
|
|||
JSONObject all = new JSONObject();
|
||||
all.put(ALL, array);
|
||||
|
||||
if(!connectorBundles.isEmpty()) {
|
||||
if(!connectorConfigBundles.isEmpty()) {
|
||||
JSONObject bundles = new JSONObject();
|
||||
|
||||
for(Map.Entry<Long, ResourceBundle> entry : connectorBundles.entrySet()) {
|
||||
for(Map.Entry<Long, ResourceBundle> entry : connectorConfigBundles.entrySet()) {
|
||||
bundles.put(entry.getKey().toString(),
|
||||
extractResourceBundle(entry.getValue()));
|
||||
}
|
||||
|
||||
all.put(CONNECTOR_RESOURCES, bundles);
|
||||
all.put(CONNECTOR_CONFIGS, bundles);
|
||||
}
|
||||
if(frameworkBundle != null) {
|
||||
all.put(FRAMEWORK_RESOURCES,extractResourceBundle(frameworkBundle));
|
||||
if(driverConfigBundle != null) {
|
||||
all.put(DRIVER_CONFIGS,extractResourceBundle(driverConfigBundle));
|
||||
}
|
||||
return all;
|
||||
}
|
||||
|
@ -140,7 +139,7 @@ public class ConnectionBean implements JsonBean {
|
|||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public void restore(JSONObject jsonObject) {
|
||||
connections = new ArrayList<MConnection>();
|
||||
links = new ArrayList<MLink>();
|
||||
|
||||
JSONArray array = (JSONArray) jsonObject.get(ALL);
|
||||
|
||||
|
@ -154,32 +153,32 @@ public class ConnectionBean implements JsonBean {
|
|||
List<MForm> connectorForms = restoreForms(connectorPart);
|
||||
List<MForm> frameworkForms = restoreForms(frameworkPart);
|
||||
|
||||
MConnection connection = new MConnection(connectorId,
|
||||
MLink link = new MLink(connectorId,
|
||||
new MConnectionForms(connectorForms),
|
||||
new MConnectionForms(frameworkForms));
|
||||
|
||||
connection.setPersistenceId((Long) object.get(ID));
|
||||
connection.setName((String) object.get(NAME));
|
||||
connection.setEnabled((Boolean) object.get(ENABLED));
|
||||
connection.setCreationUser((String) object.get(CREATION_USER));
|
||||
connection.setCreationDate(new Date((Long) object.get(CREATION_DATE)));
|
||||
connection.setLastUpdateUser((String) object.get(UPDATE_USER));
|
||||
connection.setLastUpdateDate(new Date((Long) object.get(UPDATE_DATE)));
|
||||
link.setPersistenceId((Long) object.get(ID));
|
||||
link.setName((String) object.get(NAME));
|
||||
link.setEnabled((Boolean) object.get(ENABLED));
|
||||
link.setCreationUser((String) object.get(CREATION_USER));
|
||||
link.setCreationDate(new Date((Long) object.get(CREATION_DATE)));
|
||||
link.setLastUpdateUser((String) object.get(UPDATE_USER));
|
||||
link.setLastUpdateDate(new Date((Long) object.get(UPDATE_DATE)));
|
||||
|
||||
connections.add(connection);
|
||||
links.add(link);
|
||||
}
|
||||
|
||||
if(jsonObject.containsKey(CONNECTOR_RESOURCES)) {
|
||||
JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_RESOURCES);
|
||||
if(jsonObject.containsKey(CONNECTOR_CONFIGS)) {
|
||||
JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_CONFIGS);
|
||||
Set<Map.Entry<String, JSONObject>> entrySet = bundles.entrySet();
|
||||
for (Map.Entry<String, JSONObject> entry : entrySet) {
|
||||
connectorBundles.put(Long.parseLong(entry.getKey()),
|
||||
connectorConfigBundles.put(Long.parseLong(entry.getKey()),
|
||||
restoreResourceBundle(entry.getValue()));
|
||||
}
|
||||
}
|
||||
if(jsonObject.containsKey(FRAMEWORK_RESOURCES)) {
|
||||
frameworkBundle = restoreResourceBundle(
|
||||
(JSONObject) jsonObject.get(FRAMEWORK_RESOURCES));
|
||||
if(jsonObject.containsKey(DRIVER_CONFIGS)) {
|
||||
driverConfigBundle = restoreResourceBundle(
|
||||
(JSONObject) jsonObject.get(DRIVER_CONFIGS));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -30,7 +30,7 @@ import java.util.Map;
|
|||
* part of validated entity. Optionally validation bean can also transfer
|
||||
* created persistent id in case that new entity was created.
|
||||
*/
|
||||
public class ConnectionValidationBean implements JsonBean {
|
||||
public class LinkValidationBean implements JsonBean {
|
||||
|
||||
private static final String ID = "id";
|
||||
private static final String FRAMEWORK = "framework";
|
||||
|
@ -44,7 +44,7 @@ public class ConnectionValidationBean implements JsonBean {
|
|||
private Validation frameworkValidation;
|
||||
|
||||
// For "extract"
|
||||
public ConnectionValidationBean(Validation connector, Validation framework) {
|
||||
public LinkValidationBean(Validation connector, Validation framework) {
|
||||
this();
|
||||
|
||||
this.connectorValidation = connector;
|
||||
|
@ -52,7 +52,7 @@ public class ConnectionValidationBean implements JsonBean {
|
|||
}
|
||||
|
||||
// For "restore"
|
||||
public ConnectionValidationBean() {
|
||||
public LinkValidationBean() {
|
||||
id = null;
|
||||
}
|
||||
|
|
@ -44,7 +44,7 @@ public final class FormSerialization {
|
|||
public static final String ID = "id";
|
||||
public static final String NAME = "name";
|
||||
public static final String VERSION = "version";
|
||||
public static final String FRAMEWORK_VERSION = "framework-version";
|
||||
public static final String DRIVER_VERSION = "driver-version";
|
||||
public static final String CLASS = "class";
|
||||
public static final String ENABLED = "enabled";
|
||||
public static final String CREATION_USER = "creation-user";
|
||||
|
|
|
@ -33,9 +33,9 @@ import java.util.ResourceBundle;
|
|||
*/
|
||||
public final class ResourceBundleSerialization {
|
||||
|
||||
public static final String RESOURCES = "resources";
|
||||
public static final String CONNECTOR_RESOURCES = "resources-connector";
|
||||
public static final String FRAMEWORK_RESOURCES = "resources-framework";
|
||||
public static final String CONFIGS = "configs";
|
||||
public static final String CONNECTOR_CONFIGS = "connector-configs";
|
||||
public static final String DRIVER_CONFIGS = "driver-configs";
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JSONArray extractResourceBundles(List<ResourceBundle> bundles) {
|
||||
|
@ -50,27 +50,20 @@ public final class ResourceBundleSerialization {
|
|||
@SuppressWarnings("unchecked")
|
||||
public static JSONObject extractResourceBundle(ResourceBundle bundle) {
|
||||
JSONObject json = new JSONObject();
|
||||
|
||||
Enumeration<String> keys = bundle.getKeys();
|
||||
|
||||
while(keys.hasMoreElements()) {
|
||||
String key = keys.nextElement();
|
||||
|
||||
json.put(key, bundle.getString(key));
|
||||
|
||||
}
|
||||
|
||||
return json;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static List<ResourceBundle> restoreResourceBundles(JSONArray array) {
|
||||
List<ResourceBundle> bundles = new LinkedList<ResourceBundle>();
|
||||
|
||||
for (Object item : array) {
|
||||
bundles.add(restoreResourceBundle((JSONObject) item));
|
||||
}
|
||||
|
||||
return bundles;
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import java.lang.annotation.Target;
|
|||
@Target(ElementType.FIELD)
|
||||
public @interface Input {
|
||||
/**
|
||||
* Sqoop framework will ensure that sensitive information will not be easily
|
||||
* Sqoop will ensure that sensitive information will not be easily
|
||||
* accessible.
|
||||
*
|
||||
* @return True if field is sensitive
|
||||
|
|
|
@ -18,25 +18,23 @@
|
|||
package org.apache.sqoop.model;
|
||||
|
||||
/**
|
||||
* Metadata describing framework options for connection and jobForms for each
|
||||
* supported jobForms type.
|
||||
* Describes the configs associated with the {@link Driver} for executing sqoop jobs.
|
||||
*/
|
||||
public class MFramework extends MPersistableEntity implements MClonable {
|
||||
public class MDriverConfig extends MPersistableEntity implements MClonable {
|
||||
|
||||
private final MConnectionForms connectionForms;
|
||||
private final MJobForms jobForms;
|
||||
String version;
|
||||
|
||||
public MFramework(MConnectionForms connectionForms, MJobForms jobForms,
|
||||
String version) {
|
||||
this.version = version;
|
||||
public MDriverConfig(MConnectionForms connectionForms, MJobForms jobForms, String version) {
|
||||
this.connectionForms = connectionForms;
|
||||
this.jobForms = jobForms;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder("framework-");
|
||||
StringBuilder sb = new StringBuilder("driver-");
|
||||
sb.append(getPersistenceId()).append(":");
|
||||
sb.append("version = " + version);
|
||||
sb.append(", ").append(connectionForms.toString());
|
||||
|
@ -51,11 +49,11 @@ public class MFramework extends MPersistableEntity implements MClonable {
|
|||
return true;
|
||||
}
|
||||
|
||||
if (!(other instanceof MFramework)) {
|
||||
if (!(other instanceof MDriverConfig)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
MFramework mo = (MFramework) other;
|
||||
MDriverConfig mo = (MDriverConfig) other;
|
||||
return version.equals(mo.getVersion()) &&
|
||||
connectionForms.equals(mo.connectionForms) &&
|
||||
jobForms.equals(mo.jobForms);
|
||||
|
@ -78,10 +76,10 @@ public class MFramework extends MPersistableEntity implements MClonable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public MFramework clone(boolean cloneWithValue) {
|
||||
public MDriverConfig clone(boolean cloneWithValue) {
|
||||
//Framework never have any values filled
|
||||
cloneWithValue = false;
|
||||
MFramework copy = new MFramework(this.getConnectionForms().clone(cloneWithValue),
|
||||
MDriverConfig copy = new MDriverConfig(this.getConnectionForms().clone(cloneWithValue),
|
||||
this.getJobForms().clone(cloneWithValue), this.version);
|
||||
copy.setPersistenceId(this.getPersistenceId());
|
||||
return copy;
|
|
@ -18,7 +18,7 @@
|
|||
package org.apache.sqoop.model;
|
||||
|
||||
/**
|
||||
* Represents a parameter input used by the connector for creating a connection
|
||||
* Represents a parameter input used by the connector for creating a link
|
||||
* or a job object.
|
||||
* @param <T> the value type associated with this parameter
|
||||
* @param boolean whether or not the field contains sensitive information
|
||||
|
|
|
@ -30,17 +30,17 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
* Connector reference.
|
||||
*
|
||||
* Job object do not immediately depend on connector as there is indirect
|
||||
* dependency through connection object, but having this dependency explicitly
|
||||
* dependency through link object, but having this dependency explicitly
|
||||
* carried along helps a lot.
|
||||
*/
|
||||
private final long fromConnectorId;
|
||||
private final long toConnectorId;
|
||||
|
||||
/**
|
||||
* Corresponding connection objects for connector.
|
||||
* Corresponding link objects for connector.
|
||||
*/
|
||||
private final long fromConnectionId;
|
||||
private final long toConnectionId;
|
||||
private final long fromLinkId;
|
||||
private final long toLinkId;
|
||||
|
||||
private final MJobForms fromConnectorPart;
|
||||
private final MJobForms toConnectorPart;
|
||||
|
@ -51,8 +51,8 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
*
|
||||
* @param fromConnectorId FROM Connector id
|
||||
* @param toConnectorId TO Connector id
|
||||
* @param fromConnectionId FROM Connection id
|
||||
* @param toConnectionId TO Connection id
|
||||
* @param fromLinkId FROM Link id
|
||||
* @param toLinkId TO Link id
|
||||
* @param fromPart FROM Connector forms
|
||||
* @param toPart TO Connector forms
|
||||
* @param frameworkPart Framework forms
|
||||
|
@ -66,8 +66,8 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
MJobForms frameworkPart) {
|
||||
this.fromConnectorId = fromConnectorId;
|
||||
this.toConnectorId = toConnectorId;
|
||||
this.fromConnectionId = fromConnectionId;
|
||||
this.toConnectionId = toConnectionId;
|
||||
this.fromLinkId = fromConnectionId;
|
||||
this.toLinkId = toConnectionId;
|
||||
this.fromConnectorPart = fromPart;
|
||||
this.toConnectorPart = toPart;
|
||||
this.frameworkPart = frameworkPart;
|
||||
|
@ -101,8 +101,8 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
|
||||
this.fromConnectorId = other.getConnectorId(Direction.FROM);
|
||||
this.toConnectorId = other.getConnectorId(Direction.TO);
|
||||
this.fromConnectionId = other.getConnectionId(Direction.FROM);
|
||||
this.toConnectionId = other.getConnectionId(Direction.TO);
|
||||
this.fromLinkId = other.getLinkId(Direction.FROM);
|
||||
this.toLinkId = other.getLinkId(Direction.TO);
|
||||
this.fromConnectorPart = fromPart;
|
||||
this.toConnectorPart = toPart;
|
||||
this.frameworkPart = frameworkPart;
|
||||
|
@ -119,13 +119,13 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
return sb.toString();
|
||||
}
|
||||
|
||||
public long getConnectionId(Direction type) {
|
||||
public long getLinkId(Direction type) {
|
||||
switch(type) {
|
||||
case FROM:
|
||||
return fromConnectionId;
|
||||
return fromLinkId;
|
||||
|
||||
case TO:
|
||||
return toConnectionId;
|
||||
return toLinkId;
|
||||
|
||||
default:
|
||||
throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
|
||||
|
@ -170,8 +170,8 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
return new MJob(
|
||||
getConnectorId(Direction.FROM),
|
||||
getConnectorId(Direction.TO),
|
||||
getConnectionId(Direction.FROM),
|
||||
getConnectionId(Direction.TO),
|
||||
getLinkId(Direction.FROM),
|
||||
getLinkId(Direction.TO),
|
||||
getConnectorPart(Direction.FROM).clone(false),
|
||||
getConnectorPart(Direction.TO).clone(false),
|
||||
frameworkPart.clone(false));
|
||||
|
@ -191,8 +191,8 @@ public class MJob extends MAccountableEntity implements MClonable {
|
|||
MJob job = (MJob)object;
|
||||
return (job.getConnectorId(Direction.FROM) == this.getConnectorId(Direction.FROM))
|
||||
&& (job.getConnectorId(Direction.TO) == this.getConnectorId(Direction.TO))
|
||||
&& (job.getConnectionId(Direction.FROM) == this.getConnectionId(Direction.FROM))
|
||||
&& (job.getConnectionId(Direction.TO) == this.getConnectionId(Direction.TO))
|
||||
&& (job.getLinkId(Direction.FROM) == this.getLinkId(Direction.FROM))
|
||||
&& (job.getLinkId(Direction.TO) == this.getLinkId(Direction.TO))
|
||||
&& (job.getPersistenceId() == this.getPersistenceId())
|
||||
&& (job.getConnectorPart(Direction.FROM).equals(this.getConnectorPart(Direction.FROM)))
|
||||
&& (job.getConnectorPart(Direction.TO).equals(this.getConnectorPart(Direction.TO)))
|
||||
|
|
|
@ -18,10 +18,9 @@
|
|||
package org.apache.sqoop.model;
|
||||
|
||||
/**
|
||||
* Model describing entire connection object including both connector and
|
||||
* framework part.
|
||||
* Model describing the link object and its corresponding configs
|
||||
*/
|
||||
public class MConnection extends MAccountableEntity implements MClonable {
|
||||
public class MLink extends MAccountableEntity implements MClonable {
|
||||
private long connectorId;
|
||||
|
||||
private final MConnectionForms connectorPart;
|
||||
|
@ -34,7 +33,7 @@ public class MConnection extends MAccountableEntity implements MClonable {
|
|||
* @param connectorPart Connector forms
|
||||
* @param frameworkPart Framework forms
|
||||
*/
|
||||
public MConnection(long connectorId,
|
||||
public MLink(long connectorId,
|
||||
MConnectionForms connectorPart,
|
||||
MConnectionForms frameworkPart) {
|
||||
this.connectorId = connectorId;
|
||||
|
@ -47,7 +46,7 @@ public class MConnection extends MAccountableEntity implements MClonable {
|
|||
*
|
||||
* @param other MConnection model to copy
|
||||
*/
|
||||
public MConnection(MConnection other) {
|
||||
public MLink(MLink other) {
|
||||
this(other, other.connectorPart.clone(true), other.frameworkPart.clone(true));
|
||||
}
|
||||
|
||||
|
@ -61,7 +60,7 @@ public class MConnection extends MAccountableEntity implements MClonable {
|
|||
* @param connectorPart Connector forms
|
||||
* @param frameworkPart Framework forms
|
||||
*/
|
||||
public MConnection(MConnection other, MConnectionForms connectorPart, MConnectionForms frameworkPart) {
|
||||
public MLink(MLink other, MConnectionForms connectorPart, MConnectionForms frameworkPart) {
|
||||
super(other);
|
||||
this.connectorId = other.connectorId;
|
||||
this.connectorPart = connectorPart;
|
||||
|
@ -103,11 +102,11 @@ public class MConnection extends MAccountableEntity implements MClonable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public MConnection clone(boolean cloneWithValue) {
|
||||
public MLink clone(boolean cloneWithValue) {
|
||||
if(cloneWithValue) {
|
||||
return new MConnection(this);
|
||||
return new MLink(this);
|
||||
} else {
|
||||
return new MConnection(connectorId, connectorPart.clone(false), frameworkPart.clone(false));
|
||||
return new MLink(connectorId, connectorPart.clone(false), frameworkPart.clone(false));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -117,11 +116,11 @@ public class MConnection extends MAccountableEntity implements MClonable {
|
|||
return true;
|
||||
}
|
||||
|
||||
if(!(object instanceof MConnection)) {
|
||||
if(!(object instanceof MLink)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
MConnection mc = (MConnection)object;
|
||||
MLink mc = (MLink)object;
|
||||
return (mc.connectorId == this.connectorId)
|
||||
&& (mc.getPersistenceId() == this.getPersistenceId())
|
||||
&& (mc.connectorPart.equals(this.connectorPart))
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
package org.apache.sqoop.json;
|
||||
|
||||
import org.apache.sqoop.model.MFramework;
|
||||
import org.apache.sqoop.model.MDriverConfig;
|
||||
import org.json.simple.JSONObject;
|
||||
import org.json.simple.JSONValue;
|
||||
import org.junit.Test;
|
||||
|
@ -32,7 +32,7 @@ import static org.junit.Assert.*;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class TestFrameworkBean {
|
||||
public class TestDriverConfigBean {
|
||||
|
||||
/**
|
||||
* Test that by JSON serialization followed by deserialization we will get
|
||||
|
@ -40,10 +40,10 @@ public class TestFrameworkBean {
|
|||
*/
|
||||
@Test
|
||||
public void testSerialization() {
|
||||
MFramework framework = getFramework();
|
||||
MDriverConfig driverConfig = getDriverConfig();
|
||||
|
||||
// Serialize it to JSON object
|
||||
FrameworkBean bean = new FrameworkBean(framework, getResourceBundle());
|
||||
DriverConfigBean bean = new DriverConfigBean(driverConfig, getResourceBundle());
|
||||
JSONObject json = bean.extract(false);
|
||||
|
||||
// "Move" it across network in text form
|
||||
|
@ -51,10 +51,10 @@ public class TestFrameworkBean {
|
|||
|
||||
// Retrieved transferred object
|
||||
JSONObject retrievedJson = (JSONObject) JSONValue.parse(string);
|
||||
FrameworkBean retrievedBean = new FrameworkBean();
|
||||
DriverConfigBean retrievedBean = new DriverConfigBean();
|
||||
retrievedBean.restore(retrievedJson);
|
||||
|
||||
assertEquals(framework, retrievedBean.getFramework());
|
||||
assertEquals(driverConfig, retrievedBean.getDriverConfig());
|
||||
|
||||
ResourceBundle retrievedBundle = retrievedBean.getResourceBundle();
|
||||
assertEquals("a", retrievedBundle.getString("a"));
|
|
@ -68,8 +68,8 @@ public class TestJobBean {
|
|||
|
||||
// Check id and name
|
||||
assertEquals(666, target.getPersistenceId());
|
||||
assertEquals(target.getConnectionId(Direction.FROM), 1);
|
||||
assertEquals(target.getConnectionId(Direction.TO), 2);
|
||||
assertEquals(target.getLinkId(Direction.FROM), 1);
|
||||
assertEquals(target.getLinkId(Direction.TO), 2);
|
||||
assertEquals(target.getConnectorId(Direction.FROM), 1);
|
||||
assertEquals(target.getConnectorId(Direction.TO), 2);
|
||||
assertEquals("The big job", target.getName());
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
package org.apache.sqoop.json;
|
||||
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.apache.sqoop.model.MStringInput;
|
||||
import org.json.simple.JSONArray;
|
||||
import org.json.simple.JSONObject;
|
||||
|
@ -33,27 +33,27 @@ import static org.apache.sqoop.json.TestUtil.*;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class TestConnectionBean {
|
||||
public class TestLinkBean {
|
||||
@Test
|
||||
public void testSerialization() {
|
||||
Date created = new Date();
|
||||
Date updated = new Date();
|
||||
MConnection connection = getConnection("ahoj");
|
||||
connection.setName("Connection");
|
||||
connection.setPersistenceId(666);
|
||||
connection.setCreationUser("admin");
|
||||
connection.setCreationDate(created);
|
||||
connection.setLastUpdateUser("user");
|
||||
connection.setLastUpdateDate(updated);
|
||||
connection.setEnabled(false);
|
||||
MLink link = getLink("ahoj");
|
||||
link.setName("Connection");
|
||||
link.setPersistenceId(666);
|
||||
link.setCreationUser("admin");
|
||||
link.setCreationDate(created);
|
||||
link.setLastUpdateUser("user");
|
||||
link.setLastUpdateDate(updated);
|
||||
link.setEnabled(false);
|
||||
|
||||
// Fill some data at the beginning
|
||||
MStringInput input = (MStringInput) connection.getConnectorPart().getForms()
|
||||
MStringInput input = (MStringInput) link.getConnectorPart().getForms()
|
||||
.get(0).getInputs().get(0);
|
||||
input.setValue("Hi there!");
|
||||
|
||||
// Serialize it to JSON object
|
||||
ConnectionBean bean = new ConnectionBean(connection);
|
||||
LinkBean bean = new LinkBean(link);
|
||||
JSONObject json = bean.extract(false);
|
||||
|
||||
// Check for sensitivity
|
||||
|
@ -71,9 +71,9 @@ public class TestConnectionBean {
|
|||
|
||||
// Retrieved transferred object
|
||||
JSONObject retrievedJson = (JSONObject) JSONValue.parse(string);
|
||||
ConnectionBean retrievedBean = new ConnectionBean();
|
||||
LinkBean retrievedBean = new LinkBean();
|
||||
retrievedBean.restore(retrievedJson);
|
||||
MConnection target = retrievedBean.getConnections().get(0);
|
||||
MLink target = retrievedBean.getLinks().get(0);
|
||||
|
||||
// Check id and name
|
||||
assertEquals(666, target.getPersistenceId());
|
||||
|
@ -94,22 +94,22 @@ public class TestConnectionBean {
|
|||
public void testSensitivityFilter() {
|
||||
Date created = new Date();
|
||||
Date updated = new Date();
|
||||
MConnection connection = getConnection("ahoj");
|
||||
connection.setName("Connection");
|
||||
connection.setPersistenceId(666);
|
||||
connection.setCreationUser("admin");
|
||||
connection.setCreationDate(created);
|
||||
connection.setLastUpdateUser("user");
|
||||
connection.setLastUpdateDate(updated);
|
||||
connection.setEnabled(true);
|
||||
MLink link = getLink("ahoj");
|
||||
link.setName("Connection");
|
||||
link.setPersistenceId(666);
|
||||
link.setCreationUser("admin");
|
||||
link.setCreationDate(created);
|
||||
link.setLastUpdateUser("user");
|
||||
link.setLastUpdateDate(updated);
|
||||
link.setEnabled(true);
|
||||
|
||||
// Fill some data at the beginning
|
||||
MStringInput input = (MStringInput) connection.getConnectorPart().getForms()
|
||||
MStringInput input = (MStringInput) link.getConnectorPart().getForms()
|
||||
.get(0).getInputs().get(0);
|
||||
input.setValue("Hi there!");
|
||||
|
||||
// Serialize it to JSON object
|
||||
ConnectionBean bean = new ConnectionBean(connection);
|
||||
LinkBean bean = new LinkBean(link);
|
||||
JSONObject json = bean.extract(false);
|
||||
JSONObject jsonFiltered = bean.extract(true);
|
||||
|
||||
|
@ -120,7 +120,7 @@ public class TestConnectionBean {
|
|||
JSONObject connector = (JSONObject)connectors.get(0);
|
||||
JSONArray inputs = (JSONArray)connector.get("inputs");
|
||||
assertEquals(3, inputs.size());
|
||||
// Inputs are ordered when creating connection
|
||||
// Inputs are ordered when creating link
|
||||
JSONObject password = (JSONObject)inputs.get(2);
|
||||
assertTrue(password.containsKey("value"));
|
||||
|
||||
|
@ -131,7 +131,7 @@ public class TestConnectionBean {
|
|||
connector = (JSONObject)connectors.get(0);
|
||||
inputs = (JSONArray)connector.get("inputs");
|
||||
assertEquals(3, inputs.size());
|
||||
// Inputs are ordered when creating connection
|
||||
// Inputs are ordered when creating link
|
||||
password = (JSONObject)inputs.get(2);
|
||||
assertFalse(password.containsKey("value"));
|
||||
}
|
|
@ -18,11 +18,11 @@
|
|||
package org.apache.sqoop.json;
|
||||
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MConnector;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.apache.sqoop.model.MFramework;
|
||||
import org.apache.sqoop.model.MDriverConfig;
|
||||
import org.apache.sqoop.model.MInput;
|
||||
import org.apache.sqoop.model.MJob;
|
||||
import org.apache.sqoop.model.MJobForms;
|
||||
|
@ -44,23 +44,18 @@ public class TestUtil {
|
|||
getConnectionForms(), getJobForms(), getJobForms());
|
||||
}
|
||||
|
||||
public static MFramework getFramework() {
|
||||
return new MFramework(getConnectionForms(), getJobForms(), "1");
|
||||
public static MDriverConfig getDriverConfig() {
|
||||
return new MDriverConfig(getConnectionForms(), getJobForms(), "1");
|
||||
}
|
||||
|
||||
public static MConnection getConnection(String name) {
|
||||
return new MConnection(1,
|
||||
getConnector(name).getConnectionForms(),
|
||||
getFramework().getConnectionForms()
|
||||
);
|
||||
public static MLink getLink(String name) {
|
||||
return new MLink(1, getConnector(name).getConnectionForms(), getDriverConfig()
|
||||
.getConnectionForms());
|
||||
}
|
||||
|
||||
public static MJob getJob(String name) {
|
||||
return new MJob(1, 2, 1, 2,
|
||||
getConnector(name).getJobForms(Direction.FROM),
|
||||
getConnector(name).getJobForms(Direction.TO),
|
||||
getFramework().getJobForms()
|
||||
);
|
||||
return new MJob(1, 2, 1, 2, getConnector(name).getJobForms(Direction.FROM), getConnector(name)
|
||||
.getJobForms(Direction.TO), getDriverConfig().getJobForms());
|
||||
}
|
||||
|
||||
public static MConnectionForms getConnectionForms() {
|
||||
|
|
|
@ -102,9 +102,9 @@ public class TestValidationBean {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testConnectionValidationBeanSerialization() {
|
||||
public void testLinkValidationBeanSerialization() {
|
||||
// Serialize it to JSON object
|
||||
ConnectionValidationBean bean = new ConnectionValidationBean(
|
||||
LinkValidationBean bean = new LinkValidationBean(
|
||||
getValidation(Status.FINE),
|
||||
getValidation(Status.UNACCEPTABLE)
|
||||
);
|
||||
|
@ -115,7 +115,7 @@ public class TestValidationBean {
|
|||
|
||||
// Retrieved transferred object
|
||||
JSONObject retrievedJson = (JSONObject) JSONValue.parse(string);
|
||||
ConnectionValidationBean retrievedBean = new ConnectionValidationBean();
|
||||
LinkValidationBean retrievedBean = new LinkValidationBean();
|
||||
retrievedBean.restore(retrievedJson);
|
||||
|
||||
assertNull(retrievedBean.getId());
|
||||
|
@ -139,9 +139,9 @@ public class TestValidationBean {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testConnectionValidationBeanId() {
|
||||
public void testLinkValidationBeanId() {
|
||||
// Serialize it to JSON object
|
||||
ConnectionValidationBean bean = new ConnectionValidationBean(
|
||||
LinkValidationBean bean = new LinkValidationBean(
|
||||
getValidation(Status.FINE),
|
||||
getValidation(Status.FINE)
|
||||
);
|
||||
|
@ -153,7 +153,7 @@ public class TestValidationBean {
|
|||
|
||||
// Retrieved transferred object
|
||||
JSONObject retrievedJson = (JSONObject) JSONValue.parse(string);
|
||||
ConnectionValidationBean retrievedBean = new ConnectionValidationBean();
|
||||
LinkValidationBean retrievedBean = new LinkValidationBean();
|
||||
retrievedBean.restore(retrievedJson);
|
||||
|
||||
assertEquals((Long)(long) 10, retrievedBean.getId());
|
||||
|
|
|
@ -41,21 +41,21 @@ public class TestMAccountableEntity {
|
|||
list.add(input);
|
||||
MForm form = new MForm("FORMNAME", list);
|
||||
forms.add(form);
|
||||
MAccountableEntity connection = new MConnection(123l, new MConnectionForms(
|
||||
MAccountableEntity link = new MLink(123l, new MConnectionForms(
|
||||
forms), new MConnectionForms(forms));
|
||||
// Initially creation date and last update date is same
|
||||
assertEquals(connection.getCreationDate(), connection.getLastUpdateDate());
|
||||
assertEquals(link.getCreationDate(), link.getLastUpdateDate());
|
||||
Date testCreationDate = new Date();
|
||||
Date testLastUpdateDate = new Date();
|
||||
connection.setCreationUser("admin");
|
||||
connection.setCreationDate(testCreationDate);
|
||||
connection.setLastUpdateUser("user");
|
||||
connection.setLastUpdateDate(testLastUpdateDate);
|
||||
connection.setEnabled(false);
|
||||
assertEquals(testCreationDate, connection.getCreationDate());
|
||||
assertEquals("admin", connection.getCreationUser());
|
||||
assertEquals(testLastUpdateDate, connection.getLastUpdateDate());
|
||||
assertEquals(false, connection.getEnabled());
|
||||
assertEquals("user", connection.getLastUpdateUser());
|
||||
link.setCreationUser("admin");
|
||||
link.setCreationDate(testCreationDate);
|
||||
link.setLastUpdateUser("user");
|
||||
link.setLastUpdateDate(testLastUpdateDate);
|
||||
link.setEnabled(false);
|
||||
assertEquals(testCreationDate, link.getCreationDate());
|
||||
assertEquals("admin", link.getCreationUser());
|
||||
assertEquals(testLastUpdateDate, link.getLastUpdateDate());
|
||||
assertEquals(false, link.getEnabled());
|
||||
assertEquals("user", link.getLastUpdateUser());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,7 @@ import static org.junit.Assert.*;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class TestMFramework {
|
||||
public class TestMDriverConfig {
|
||||
|
||||
@Test
|
||||
public void testForms() {
|
||||
|
@ -38,10 +38,10 @@ public class TestMFramework {
|
|||
MConnectionForms connectionForms = new MConnectionForms(connectionFormList);
|
||||
MJobForms jobForms = new MJobForms(jobFormList);
|
||||
|
||||
MFramework framework = new MFramework(connectionForms, jobForms, "1");
|
||||
assertEquals(1, framework.getJobForms().getForms().size());
|
||||
assertEquals("job-test", framework.getJobForms().getForms().get(0).getName());
|
||||
assertEquals(1, framework.getConnectionForms().getForms().size());
|
||||
assertEquals("connection-test", framework.getConnectionForms().getForms().get(0).getName());
|
||||
MDriverConfig driver = new MDriverConfig(connectionForms, jobForms, "1");
|
||||
assertEquals(1, driver.getJobForms().getForms().size());
|
||||
assertEquals("job-test", driver.getJobForms().getForms().get(0).getName());
|
||||
assertEquals(1, driver.getConnectionForms().getForms().size());
|
||||
assertEquals("connection-test", driver.getConnectionForms().getForms().get(0).getName());
|
||||
}
|
||||
}
|
|
@ -27,7 +27,7 @@ import org.junit.Test;
|
|||
/**
|
||||
* Test class for org.apache.sqoop.model.MConnection
|
||||
*/
|
||||
public class TestMConnection {
|
||||
public class TestMLink {
|
||||
|
||||
/**
|
||||
* Test for initialization
|
||||
|
@ -35,39 +35,39 @@ public class TestMConnection {
|
|||
@Test
|
||||
public void testInitialization() {
|
||||
// Test default constructor
|
||||
MConnection connection = connection();
|
||||
assertEquals(123l, connection.getConnectorId());
|
||||
assertEquals("Vampire", connection.getName());
|
||||
assertEquals("Buffy", connection.getCreationUser());
|
||||
assertEquals(forms1(), connection.getConnectorPart());
|
||||
assertEquals(forms2(), connection.getFrameworkPart());
|
||||
MLink link = link();
|
||||
assertEquals(123l, link.getConnectorId());
|
||||
assertEquals("Vampire", link.getName());
|
||||
assertEquals("Buffy", link.getCreationUser());
|
||||
assertEquals(forms1(), link.getConnectorPart());
|
||||
assertEquals(forms2(), link.getFrameworkPart());
|
||||
|
||||
// Test copy constructor
|
||||
MConnection copy = new MConnection(connection);
|
||||
MLink copy = new MLink(link);
|
||||
assertEquals(123l, copy.getConnectorId());
|
||||
assertEquals("Vampire", copy.getName());
|
||||
assertEquals("Buffy", copy.getCreationUser());
|
||||
assertEquals(connection.getCreationDate(), copy.getCreationDate());
|
||||
assertEquals(link.getCreationDate(), copy.getCreationDate());
|
||||
assertEquals(forms1(), copy.getConnectorPart());
|
||||
assertEquals(forms2(), copy.getFrameworkPart());
|
||||
|
||||
// Test constructor for metadata upgrade (the order of forms is different)
|
||||
MConnection upgradeCopy = new MConnection(connection, forms2(), forms1());
|
||||
MLink upgradeCopy = new MLink(link, forms2(), forms1());
|
||||
assertEquals(123l, upgradeCopy.getConnectorId());
|
||||
assertEquals("Vampire", upgradeCopy.getName());
|
||||
assertEquals("Buffy", upgradeCopy.getCreationUser());
|
||||
assertEquals(connection.getCreationDate(), upgradeCopy.getCreationDate());
|
||||
assertEquals(link.getCreationDate(), upgradeCopy.getCreationDate());
|
||||
assertEquals(forms2(), upgradeCopy.getConnectorPart());
|
||||
assertEquals(forms1(), upgradeCopy.getFrameworkPart());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClone() {
|
||||
MConnection connection = connection();
|
||||
MLink link = link();
|
||||
|
||||
// Clone without value
|
||||
MConnection withoutValue = connection.clone(false);
|
||||
assertEquals(connection, withoutValue);
|
||||
MLink withoutValue = link.clone(false);
|
||||
assertEquals(link, withoutValue);
|
||||
assertEquals(MPersistableEntity.PERSISTANCE_ID_DEFAULT, withoutValue.getPersistenceId());
|
||||
assertNull(withoutValue.getName());
|
||||
assertNull(withoutValue.getCreationUser());
|
||||
|
@ -77,22 +77,22 @@ public class TestMConnection {
|
|||
assertNull(withoutValue.getConnectorPart().getForm("FORMNAME").getInput("STRING-INPUT").getValue());
|
||||
|
||||
// Clone with value
|
||||
MConnection withValue = connection.clone(true);
|
||||
assertEquals(connection, withValue);
|
||||
assertEquals(connection.getPersistenceId(), withValue.getPersistenceId());
|
||||
assertEquals(connection.getName(), withValue.getName());
|
||||
assertEquals(connection.getCreationUser(), withValue.getCreationUser());
|
||||
MLink withValue = link.clone(true);
|
||||
assertEquals(link, withValue);
|
||||
assertEquals(link.getPersistenceId(), withValue.getPersistenceId());
|
||||
assertEquals(link.getName(), withValue.getName());
|
||||
assertEquals(link.getCreationUser(), withValue.getCreationUser());
|
||||
assertEquals(forms1(), withValue.getConnectorPart());
|
||||
assertEquals(forms2(), withValue.getFrameworkPart());
|
||||
assertEquals(100, withValue.getConnectorPart().getForm("FORMNAME").getInput("INTEGER-INPUT").getValue());
|
||||
assertEquals("TEST-VALUE", withValue.getConnectorPart().getForm("FORMNAME").getInput("STRING-INPUT").getValue());
|
||||
}
|
||||
|
||||
private MConnection connection() {
|
||||
MConnection connection = new MConnection(123l, forms1(), forms2());
|
||||
connection.setName("Vampire");
|
||||
connection.setCreationUser("Buffy");
|
||||
return connection;
|
||||
private MLink link() {
|
||||
MLink link = new MLink(123l, forms1(), forms2());
|
||||
link.setName("Vampire");
|
||||
link.setCreationUser("Buffy");
|
||||
return link;
|
||||
}
|
||||
|
||||
private MConnectionForms forms1() {
|
|
@ -22,10 +22,10 @@ import java.util.ResourceBundle;
|
|||
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.common.VersionInfo;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||
import org.apache.sqoop.connector.spi.RepositoryUpgrader;
|
||||
import org.apache.sqoop.job.etl.From;
|
||||
import org.apache.sqoop.job.etl.To;
|
||||
import org.apache.sqoop.connector.spi.SqoopConnector;
|
||||
|
@ -65,8 +65,8 @@ public class GenericJdbcConnector extends SqoopConnector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Class getConnectionConfigurationClass() {
|
||||
return ConnectionConfiguration.class;
|
||||
public Class getLinkConfigurationClass() {
|
||||
return LinkConfiguration.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -97,8 +97,8 @@ public class GenericJdbcConnector extends SqoopConnector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public MetadataUpgrader getMetadataUpgrader() {
|
||||
return new GenericJdbcConnectorMetadataUpgrader();
|
||||
public RepositoryUpgrader getRepositoryUpgrader() {
|
||||
return new GenericJdbcConnectorUpgrader();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ public final class GenericJdbcConnectorConstants {
|
|||
|
||||
// Resource bundle name
|
||||
public static final String RESOURCE_BUNDLE_NAME =
|
||||
"generic-jdbc-connector-resources";
|
||||
"generic-jdbc-connector-config";
|
||||
|
||||
/*
|
||||
* All jdbc connector related configuration is prefixed with this:
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.sqoop.connector.jdbc;
|
|||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||
import org.apache.sqoop.connector.spi.RepositoryUpgrader;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.apache.sqoop.model.MInput;
|
||||
|
@ -30,9 +30,9 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class GenericJdbcConnectorMetadataUpgrader extends MetadataUpgrader {
|
||||
public class GenericJdbcConnectorUpgrader extends RepositoryUpgrader {
|
||||
private static final Logger LOG =
|
||||
Logger.getLogger(GenericJdbcConnectorMetadataUpgrader.class);
|
||||
Logger.getLogger(GenericJdbcConnectorUpgrader.class);
|
||||
|
||||
/*
|
||||
* For now, there is no real upgrade. So copy all data over,
|
|
@ -23,22 +23,23 @@ import java.sql.SQLException;
|
|||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.ExtractorContext;
|
||||
import org.apache.sqoop.job.etl.Extractor;
|
||||
|
||||
public class GenericJdbcExtractor extends Extractor<ConnectionConfiguration, FromJobConfiguration, GenericJdbcPartition> {
|
||||
public class GenericJdbcExtractor extends Extractor<LinkConfiguration, FromJobConfiguration, GenericJdbcPartition> {
|
||||
|
||||
public static final Logger LOG = Logger.getLogger(GenericJdbcExtractor.class);
|
||||
|
||||
private long rowsRead = 0;
|
||||
@Override
|
||||
public void extract(ExtractorContext context, ConnectionConfiguration connection, FromJobConfiguration job, GenericJdbcPartition partition) {
|
||||
String driver = connection.connection.jdbcDriver;
|
||||
String url = connection.connection.connectionString;
|
||||
String username = connection.connection.username;
|
||||
String password = connection.connection.password;
|
||||
public void extract(ExtractorContext context, LinkConfiguration linkConf,
|
||||
FromJobConfiguration fromJobConf, GenericJdbcPartition partition) {
|
||||
String driver = linkConf.link.jdbcDriver;
|
||||
String url = linkConf.link.connectionString;
|
||||
String username = linkConf.link.username;
|
||||
String password = linkConf.link.password;
|
||||
GenericJdbcExecutor executor = new GenericJdbcExecutor(driver, url, username, password);
|
||||
|
||||
String query = context.getString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_FROM_DATA_SQL);
|
||||
|
|
|
@ -18,18 +18,18 @@
|
|||
package org.apache.sqoop.connector.jdbc;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Destroyer;
|
||||
import org.apache.sqoop.job.etl.DestroyerContext;
|
||||
|
||||
public class GenericJdbcFromDestroyer extends Destroyer<ConnectionConfiguration, FromJobConfiguration> {
|
||||
public class GenericJdbcFromDestroyer extends Destroyer<LinkConfiguration, FromJobConfiguration> {
|
||||
|
||||
private static final Logger LOG =
|
||||
Logger.getLogger(GenericJdbcFromDestroyer.class);
|
||||
|
||||
@Override
|
||||
public void destroy(DestroyerContext context, ConnectionConfiguration connection, FromJobConfiguration job) {
|
||||
public void destroy(DestroyerContext context, LinkConfiguration linkConf, FromJobConfiguration fromJobConf) {
|
||||
LOG.info("Running generic JDBC connector destroyer");
|
||||
}
|
||||
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.commons.lang.StringUtils;
|
|||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.util.SqlTypesUtils;
|
||||
import org.apache.sqoop.job.Constants;
|
||||
|
@ -37,7 +37,7 @@ import org.apache.sqoop.schema.Schema;
|
|||
import org.apache.sqoop.schema.type.Column;
|
||||
import org.apache.sqoop.utils.ClassUtils;
|
||||
|
||||
public class GenericJdbcFromInitializer extends Initializer<ConnectionConfiguration, FromJobConfiguration> {
|
||||
public class GenericJdbcFromInitializer extends Initializer<LinkConfiguration, FromJobConfiguration> {
|
||||
|
||||
private static final Logger LOG =
|
||||
Logger.getLogger(GenericJdbcFromInitializer.class);
|
||||
|
@ -45,34 +45,34 @@ public class GenericJdbcFromInitializer extends Initializer<ConnectionConfigurat
|
|||
private GenericJdbcExecutor executor;
|
||||
|
||||
@Override
|
||||
public void initialize(InitializerContext context, ConnectionConfiguration connection, FromJobConfiguration job) {
|
||||
configureJdbcProperties(context.getContext(), connection, job);
|
||||
public void initialize(InitializerContext context, LinkConfiguration linkConf, FromJobConfiguration fromJobConf) {
|
||||
configureJdbcProperties(context.getContext(), linkConf, fromJobConf);
|
||||
try {
|
||||
configurePartitionProperties(context.getContext(), connection, job);
|
||||
configureTableProperties(context.getContext(), connection, job);
|
||||
configurePartitionProperties(context.getContext(), linkConf, fromJobConf);
|
||||
configureTableProperties(context.getContext(), linkConf, fromJobConf);
|
||||
} finally {
|
||||
executor.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getJars(InitializerContext context, ConnectionConfiguration connection, FromJobConfiguration job) {
|
||||
public List<String> getJars(InitializerContext context, LinkConfiguration linkConf, FromJobConfiguration fromJobConf) {
|
||||
List<String> jars = new LinkedList<String>();
|
||||
|
||||
jars.add(ClassUtils.jarForClass(connection.connection.jdbcDriver));
|
||||
jars.add(ClassUtils.jarForClass(linkConf.link.jdbcDriver));
|
||||
|
||||
return jars;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema(InitializerContext context, ConnectionConfiguration connectionConfiguration, FromJobConfiguration fromJobConfiguration) {
|
||||
configureJdbcProperties(context.getContext(), connectionConfiguration, fromJobConfiguration);
|
||||
public Schema getSchema(InitializerContext context, LinkConfiguration linkConf, FromJobConfiguration fromJobConf) {
|
||||
configureJdbcProperties(context.getContext(), linkConf, fromJobConf);
|
||||
|
||||
String schemaName = fromJobConfiguration.fromTable.tableName;
|
||||
String schemaName = fromJobConf.fromJobConfig.tableName;
|
||||
if(schemaName == null) {
|
||||
schemaName = "Query";
|
||||
} else if(fromJobConfiguration.fromTable.schemaName != null) {
|
||||
schemaName = fromJobConfiguration.fromTable.schemaName + "." + schemaName;
|
||||
} else if(fromJobConf.fromJobConfig.schemaName != null) {
|
||||
schemaName = fromJobConf.fromJobConfig.schemaName + "." + schemaName;
|
||||
}
|
||||
|
||||
Schema schema = new Schema(schemaName);
|
||||
|
@ -117,11 +117,11 @@ public class GenericJdbcFromInitializer extends Initializer<ConnectionConfigurat
|
|||
}
|
||||
}
|
||||
|
||||
private void configureJdbcProperties(MutableContext context, ConnectionConfiguration connectionConfig, FromJobConfiguration jobConfig) {
|
||||
String driver = connectionConfig.connection.jdbcDriver;
|
||||
String url = connectionConfig.connection.connectionString;
|
||||
String username = connectionConfig.connection.username;
|
||||
String password = connectionConfig.connection.password;
|
||||
private void configureJdbcProperties(MutableContext context, LinkConfiguration connectionConfig, FromJobConfiguration fromJobConfig) {
|
||||
String driver = connectionConfig.link.jdbcDriver;
|
||||
String url = connectionConfig.link.connectionString;
|
||||
String username = connectionConfig.link.username;
|
||||
String password = connectionConfig.link.password;
|
||||
|
||||
assert driver != null;
|
||||
assert url != null;
|
||||
|
@ -129,15 +129,15 @@ public class GenericJdbcFromInitializer extends Initializer<ConnectionConfigurat
|
|||
executor = new GenericJdbcExecutor(driver, url, username, password);
|
||||
}
|
||||
|
||||
private void configurePartitionProperties(MutableContext context, ConnectionConfiguration connectionConfig, FromJobConfiguration jobConfig) {
|
||||
private void configurePartitionProperties(MutableContext context, LinkConfiguration connectionConfig, FromJobConfiguration fromJobConfig) {
|
||||
// ----- configure column name -----
|
||||
|
||||
String partitionColumnName = jobConfig.fromTable.partitionColumn;
|
||||
String partitionColumnName = fromJobConfig.fromJobConfig.partitionColumn;
|
||||
|
||||
if (partitionColumnName == null) {
|
||||
// if column is not specified by the user,
|
||||
// find the primary key of the fromTable (when there is a fromTable).
|
||||
String tableName = jobConfig.fromTable.tableName;
|
||||
String tableName = fromJobConfig.fromJobConfig.tableName;
|
||||
if (tableName != null) {
|
||||
partitionColumnName = executor.getPrimaryKey(tableName);
|
||||
}
|
||||
|
@ -155,14 +155,14 @@ public class GenericJdbcFromInitializer extends Initializer<ConnectionConfigurat
|
|||
|
||||
// ----- configure column type, min value, and max value -----
|
||||
|
||||
String minMaxQuery = jobConfig.fromTable.boundaryQuery;
|
||||
String minMaxQuery = fromJobConfig.fromJobConfig.boundaryQuery;
|
||||
|
||||
if (minMaxQuery == null) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
|
||||
String schemaName = jobConfig.fromTable.schemaName;
|
||||
String tableName = jobConfig.fromTable.tableName;
|
||||
String tableSql = jobConfig.fromTable.sql;
|
||||
String schemaName = fromJobConfig.fromJobConfig.schemaName;
|
||||
String tableName = fromJobConfig.fromJobConfig.tableName;
|
||||
String tableSql = fromJobConfig.fromJobConfig.sql;
|
||||
|
||||
if (tableName != null && tableSql != null) {
|
||||
// when both fromTable name and fromTable sql are specified:
|
||||
|
@ -234,14 +234,14 @@ public class GenericJdbcFromInitializer extends Initializer<ConnectionConfigurat
|
|||
}
|
||||
}
|
||||
|
||||
private void configureTableProperties(MutableContext context, ConnectionConfiguration connectionConfig, FromJobConfiguration jobConfig) {
|
||||
private void configureTableProperties(MutableContext context, LinkConfiguration connectionConfig, FromJobConfiguration fromJobConfig) {
|
||||
String dataSql;
|
||||
String fieldNames;
|
||||
|
||||
String schemaName = jobConfig.fromTable.schemaName;
|
||||
String tableName = jobConfig.fromTable.tableName;
|
||||
String tableSql = jobConfig.fromTable.sql;
|
||||
String tableColumns = jobConfig.fromTable.columns;
|
||||
String schemaName = fromJobConfig.fromJobConfig.schemaName;
|
||||
String tableName = fromJobConfig.fromJobConfig.tableName;
|
||||
String tableSql = fromJobConfig.fromJobConfig.sql;
|
||||
String tableColumns = fromJobConfig.fromJobConfig.columns;
|
||||
|
||||
if (tableName != null && tableSql != null) {
|
||||
// when both fromTable name and fromTable sql are specified:
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
*/
|
||||
package org.apache.sqoop.connector.jdbc;
|
||||
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Loader;
|
||||
import org.apache.sqoop.job.etl.LoaderContext;
|
||||
|
||||
public class GenericJdbcLoader extends Loader<ConnectionConfiguration, ToJobConfiguration> {
|
||||
public class GenericJdbcLoader extends Loader<LinkConfiguration, ToJobConfiguration> {
|
||||
|
||||
public static final int DEFAULT_ROWS_PER_BATCH = 100;
|
||||
public static final int DEFAULT_BATCHES_PER_TRANSACTION = 100;
|
||||
|
@ -30,11 +30,11 @@ public class GenericJdbcLoader extends Loader<ConnectionConfiguration, ToJobConf
|
|||
private int batchesPerTransaction = DEFAULT_BATCHES_PER_TRANSACTION;
|
||||
|
||||
@Override
|
||||
public void load(LoaderContext context, ConnectionConfiguration connection, ToJobConfiguration job) throws Exception{
|
||||
String driver = connection.connection.jdbcDriver;
|
||||
String url = connection.connection.connectionString;
|
||||
String username = connection.connection.username;
|
||||
String password = connection.connection.password;
|
||||
public void load(LoaderContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) throws Exception{
|
||||
String driver = linkConf.link.jdbcDriver;
|
||||
String url = linkConf.link.connectionString;
|
||||
String username = linkConf.link.username;
|
||||
String password = linkConf.link.password;
|
||||
GenericJdbcExecutor executor = new GenericJdbcExecutor(driver, url, username, password);
|
||||
executor.setAutoCommit(false);
|
||||
|
||||
|
|
|
@ -28,13 +28,13 @@ import java.util.List;
|
|||
import java.util.TimeZone;
|
||||
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Partition;
|
||||
import org.apache.sqoop.job.etl.Partitioner;
|
||||
import org.apache.sqoop.job.etl.PartitionerContext;
|
||||
|
||||
public class GenericJdbcPartitioner extends Partitioner<ConnectionConfiguration, FromJobConfiguration> {
|
||||
public class GenericJdbcPartitioner extends Partitioner<LinkConfiguration, FromJobConfiguration> {
|
||||
|
||||
private static final BigDecimal NUMERIC_MIN_INCREMENT = new BigDecimal(10000 * Double.MIN_VALUE);
|
||||
|
||||
|
@ -47,7 +47,7 @@ public class GenericJdbcPartitioner extends Partitioner<ConnectionConfiguration,
|
|||
private Boolean partitionColumnNull;
|
||||
|
||||
@Override
|
||||
public List<Partition> getPartitions(PartitionerContext context,ConnectionConfiguration connection, FromJobConfiguration job) {
|
||||
public List<Partition> getPartitions(PartitionerContext context,LinkConfiguration linkConf, FromJobConfiguration fromJobConf) {
|
||||
List<Partition> partitions = new LinkedList<Partition>();
|
||||
|
||||
numberPartitions = context.getMaxPartitions();
|
||||
|
@ -56,7 +56,7 @@ public class GenericJdbcPartitioner extends Partitioner<ConnectionConfiguration,
|
|||
partitionMinValue = context.getString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE);
|
||||
partitionMaxValue = context.getString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE);
|
||||
|
||||
partitionColumnNull = job.fromTable.partitionColumnNull;
|
||||
partitionColumnNull = fromJobConf.fromJobConfig.partitionColumnNull;
|
||||
if (partitionColumnNull == null) {
|
||||
partitionColumnNull = false;
|
||||
}
|
||||
|
|
|
@ -18,36 +18,36 @@
|
|||
package org.apache.sqoop.connector.jdbc;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Destroyer;
|
||||
import org.apache.sqoop.job.etl.DestroyerContext;
|
||||
|
||||
public class GenericJdbcToDestroyer extends Destroyer<ConnectionConfiguration, ToJobConfiguration> {
|
||||
public class GenericJdbcToDestroyer extends Destroyer<LinkConfiguration, ToJobConfiguration> {
|
||||
|
||||
private static final Logger LOG = Logger.getLogger(GenericJdbcToDestroyer.class);
|
||||
|
||||
@Override
|
||||
public void destroy(DestroyerContext context, ConnectionConfiguration connection, ToJobConfiguration job) {
|
||||
public void destroy(DestroyerContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) {
|
||||
LOG.info("Running generic JDBC connector destroyer");
|
||||
|
||||
final String tableName = job.toTable.tableName;
|
||||
final String stageTableName = job.toTable.stageTableName;
|
||||
final String tableName = toJobConf.toJobConfig.tableName;
|
||||
final String stageTableName = toJobConf.toJobConfig.stageTableName;
|
||||
final boolean stageEnabled = stageTableName != null &&
|
||||
stageTableName.length() > 0;
|
||||
if(stageEnabled) {
|
||||
moveDataToDestinationTable(connection,
|
||||
moveDataToDestinationTable(linkConf,
|
||||
context.isSuccess(), stageTableName, tableName);
|
||||
}
|
||||
}
|
||||
|
||||
private void moveDataToDestinationTable(ConnectionConfiguration connectorConf,
|
||||
private void moveDataToDestinationTable(LinkConfiguration linkConf,
|
||||
boolean success, String stageTableName, String tableName) {
|
||||
GenericJdbcExecutor executor =
|
||||
new GenericJdbcExecutor(connectorConf.connection.jdbcDriver,
|
||||
connectorConf.connection.connectionString,
|
||||
connectorConf.connection.username,
|
||||
connectorConf.connection.password);
|
||||
new GenericJdbcExecutor(linkConf.link.jdbcDriver,
|
||||
linkConf.link.connectionString,
|
||||
linkConf.link.username,
|
||||
linkConf.link.password);
|
||||
try {
|
||||
if(success) {
|
||||
LOG.info("Job completed, transferring data from stage fromTable to " +
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.commons.lang.StringUtils;
|
|||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.util.SqlTypesUtils;
|
||||
import org.apache.sqoop.job.etl.Initializer;
|
||||
|
@ -36,44 +36,42 @@ import org.apache.sqoop.schema.Schema;
|
|||
import org.apache.sqoop.schema.type.Column;
|
||||
import org.apache.sqoop.utils.ClassUtils;
|
||||
|
||||
public class GenericJdbcToInitializer extends Initializer<ConnectionConfiguration, ToJobConfiguration> {
|
||||
public class GenericJdbcToInitializer extends Initializer<LinkConfiguration, ToJobConfiguration> {
|
||||
|
||||
private GenericJdbcExecutor executor;
|
||||
private static final Logger LOG =
|
||||
Logger.getLogger(GenericJdbcToInitializer.class);
|
||||
|
||||
@Override
|
||||
public void initialize(InitializerContext context, ConnectionConfiguration connection, ToJobConfiguration job) {
|
||||
configureJdbcProperties(context.getContext(), connection, job);
|
||||
public void initialize(InitializerContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) {
|
||||
configureJdbcProperties(context.getContext(), linkConf, toJobConf);
|
||||
try {
|
||||
configureTableProperties(context.getContext(), connection, job);
|
||||
configureTableProperties(context.getContext(), linkConf, toJobConf);
|
||||
} finally {
|
||||
executor.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getJars(InitializerContext context, ConnectionConfiguration connection, ToJobConfiguration job) {
|
||||
public List<String> getJars(InitializerContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) {
|
||||
List<String> jars = new LinkedList<String>();
|
||||
|
||||
jars.add(ClassUtils.jarForClass(connection.connection.jdbcDriver));
|
||||
|
||||
jars.add(ClassUtils.jarForClass(linkConf.link.jdbcDriver));
|
||||
return jars;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema(InitializerContext context, ConnectionConfiguration connectionConfiguration, ToJobConfiguration toJobConfiguration) {
|
||||
configureJdbcProperties(context.getContext(), connectionConfiguration, toJobConfiguration);
|
||||
public Schema getSchema(InitializerContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) {
|
||||
configureJdbcProperties(context.getContext(), linkConf, toJobConf);
|
||||
|
||||
String schemaName = toJobConfiguration.toTable.tableName;
|
||||
String schemaName = toJobConf.toJobConfig.tableName;
|
||||
|
||||
if (schemaName == null) {
|
||||
throw new SqoopException(GenericJdbcConnectorError.GENERIC_JDBC_CONNECTOR_0019,
|
||||
"Table name extraction not supported yet.");
|
||||
}
|
||||
|
||||
if(toJobConfiguration.toTable.schemaName != null) {
|
||||
schemaName = toJobConfiguration.toTable.schemaName + "." + schemaName;
|
||||
if(toJobConf.toJobConfig.schemaName != null) {
|
||||
schemaName = toJobConf.toJobConfig.schemaName + "." + schemaName;
|
||||
}
|
||||
|
||||
Schema schema = new Schema(schemaName);
|
||||
|
@ -112,11 +110,11 @@ public class GenericJdbcToInitializer extends Initializer<ConnectionConfiguratio
|
|||
}
|
||||
}
|
||||
|
||||
private void configureJdbcProperties(MutableContext context, ConnectionConfiguration connectionConfig, ToJobConfiguration jobConfig) {
|
||||
String driver = connectionConfig.connection.jdbcDriver;
|
||||
String url = connectionConfig.connection.connectionString;
|
||||
String username = connectionConfig.connection.username;
|
||||
String password = connectionConfig.connection.password;
|
||||
private void configureJdbcProperties(MutableContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) {
|
||||
String driver = linkConf.link.jdbcDriver;
|
||||
String url = linkConf.link.connectionString;
|
||||
String username = linkConf.link.username;
|
||||
String password = linkConf.link.password;
|
||||
|
||||
assert driver != null;
|
||||
assert url != null;
|
||||
|
@ -124,18 +122,18 @@ public class GenericJdbcToInitializer extends Initializer<ConnectionConfiguratio
|
|||
executor = new GenericJdbcExecutor(driver, url, username, password);
|
||||
}
|
||||
|
||||
private void configureTableProperties(MutableContext context, ConnectionConfiguration connectionConfig, ToJobConfiguration jobConfig) {
|
||||
private void configureTableProperties(MutableContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConfig) {
|
||||
String dataSql;
|
||||
|
||||
String schemaName = jobConfig.toTable.schemaName;
|
||||
String tableName = jobConfig.toTable.tableName;
|
||||
String stageTableName = jobConfig.toTable.stageTableName;
|
||||
boolean clearStageTable = jobConfig.toTable.clearStageTable == null ?
|
||||
false : jobConfig.toTable.clearStageTable;
|
||||
String schemaName = toJobConfig.toJobConfig.schemaName;
|
||||
String tableName = toJobConfig.toJobConfig.tableName;
|
||||
String stageTableName = toJobConfig.toJobConfig.stageTableName;
|
||||
boolean clearStageTable = toJobConfig.toJobConfig.clearStageTable == null ?
|
||||
false : toJobConfig.toJobConfig.clearStageTable;
|
||||
final boolean stageEnabled =
|
||||
stageTableName != null && stageTableName.length() > 0;
|
||||
String tableSql = jobConfig.toTable.sql;
|
||||
String tableColumns = jobConfig.toTable.columns;
|
||||
String tableSql = toJobConfig.toJobConfig.sql;
|
||||
String tableColumns = toJobConfig.toJobConfig.columns;
|
||||
|
||||
if (tableName != null && tableSql != null) {
|
||||
// when both fromTable name and fromTable sql are specified:
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
package org.apache.sqoop.connector.jdbc;
|
||||
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.validation.Status;
|
||||
|
@ -34,32 +34,32 @@ import java.sql.SQLException;
|
|||
public class GenericJdbcValidator extends Validator {
|
||||
|
||||
@Override
|
||||
public Validation validateConnection(Object configuration) {
|
||||
Validation validation = new Validation(ConnectionConfiguration.class);
|
||||
ConnectionConfiguration config = (ConnectionConfiguration)configuration;
|
||||
public Validation validateLink(Object configuration) {
|
||||
Validation validation = new Validation(LinkConfiguration.class);
|
||||
LinkConfiguration linkConf = (LinkConfiguration)configuration;
|
||||
|
||||
if(config.connection.jdbcDriver == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "connection", "jdbcDriver", "Driver can't be empty");
|
||||
if(linkConf.link.jdbcDriver == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "link", "jdbcDriver", "Driver can't be empty");
|
||||
} else {
|
||||
try {
|
||||
Class.forName(config.connection.jdbcDriver);
|
||||
Class.forName(linkConf.link.jdbcDriver);
|
||||
} catch (ClassNotFoundException e) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "connection", "jdbcDriver", "Can't load specified driver");
|
||||
validation.addMessage(Status.UNACCEPTABLE, "link", "jdbcDriver", "Can't load specified driver");
|
||||
}
|
||||
}
|
||||
|
||||
if(config.connection.connectionString == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "connection", "connectionString", "JDBC URL can't be empty");
|
||||
} else if(!config.connection.connectionString.startsWith("jdbc:")) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "connection", "connectionString", "This do not seem as a valid JDBC URL");
|
||||
if(linkConf.link.connectionString == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "link", "connectionString", "JDBC URL can't be empty");
|
||||
} else if(!linkConf.link.connectionString.startsWith("jdbc:")) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "link", "connectionString", "This do not seem as a valid JDBC URL");
|
||||
}
|
||||
|
||||
// See if we can connect to the database
|
||||
try {
|
||||
DriverManager.getConnection(config.connection.connectionString,
|
||||
config.connection.username, config.connection.password);
|
||||
DriverManager.getConnection(linkConf.link.connectionString,
|
||||
linkConf.link.username, linkConf.link.password);
|
||||
} catch (SQLException e) {
|
||||
validation.addMessage(Status.ACCEPTABLE, "connection", "Can't connect to the database with given credentials: " + e.getMessage());
|
||||
validation.addMessage(Status.ACCEPTABLE, "link", "Can't connect to the database with given credentials: " + e.getMessage());
|
||||
}
|
||||
|
||||
// Return final validation object
|
||||
|
@ -81,20 +81,20 @@ public class GenericJdbcValidator extends Validator {
|
|||
private Validation validateToJobConfiguration(ToJobConfiguration configuration) {
|
||||
Validation validation = new Validation(FromJobConfiguration.class);
|
||||
|
||||
if(configuration.toTable.tableName == null && configuration.toTable.sql == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toTable", "Either table name or SQL must be specified");
|
||||
if(configuration.toJobConfig.tableName == null && configuration.toJobConfig.sql == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toJobConfig", "Either table name or SQL must be specified");
|
||||
}
|
||||
if(configuration.toTable.tableName != null && configuration.toTable.sql != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toTable", "Both table name and SQL cannot be specified");
|
||||
if(configuration.toJobConfig.tableName != null && configuration.toJobConfig.sql != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toJobConfig", "Both table name and SQL cannot be specified");
|
||||
}
|
||||
if(configuration.toTable.tableName == null &&
|
||||
configuration.toTable.stageTableName != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toTable",
|
||||
if(configuration.toJobConfig.tableName == null &&
|
||||
configuration.toJobConfig.stageTableName != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toJobConfig",
|
||||
"Stage table name cannot be specified without specifying table name");
|
||||
}
|
||||
if(configuration.toTable.stageTableName == null &&
|
||||
configuration.toTable.clearStageTable != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toTable",
|
||||
if(configuration.toJobConfig.stageTableName == null &&
|
||||
configuration.toJobConfig.clearStageTable != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "toJobConfig",
|
||||
"Clear stage table cannot be specified without specifying name of " +
|
||||
"the stage table.");
|
||||
}
|
||||
|
@ -105,18 +105,18 @@ public class GenericJdbcValidator extends Validator {
|
|||
private Validation validateFromJobConfiguration(FromJobConfiguration configuration) {
|
||||
Validation validation = new Validation(FromJobConfiguration.class);
|
||||
|
||||
if(configuration.fromTable.tableName == null && configuration.fromTable.sql == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromTable", "Either table name or SQL must be specified");
|
||||
if(configuration.fromJobConfig.tableName == null && configuration.fromJobConfig.sql == null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromJobConfig", "Either table name or SQL must be specified");
|
||||
}
|
||||
if(configuration.fromTable.tableName != null && configuration.fromTable.sql != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromTable", "Both table name and SQL cannot be specified");
|
||||
if(configuration.fromJobConfig.tableName != null && configuration.fromJobConfig.sql != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromJobConfig", "Both table name and SQL cannot be specified");
|
||||
}
|
||||
if(configuration.fromTable.schemaName != null && configuration.fromTable.sql != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromTable", "Both schema name and SQL cannot be specified");
|
||||
if(configuration.fromJobConfig.schemaName != null && configuration.fromJobConfig.sql != null) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromJobConfig", "Both schema name and SQL cannot be specified");
|
||||
}
|
||||
|
||||
if(configuration.fromTable.sql != null && !configuration.fromTable.sql.contains(GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN)) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromTable", "sql", "SQL statement must contain placeholder for auto generated "
|
||||
if(configuration.fromJobConfig.sql != null && !configuration.fromJobConfig.sql.contains(GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN)) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "fromJobConfig", "sql", "SQL statement must contain placeholder for auto generated "
|
||||
+ "conditions - " + GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN);
|
||||
}
|
||||
|
||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.sqoop.validation.validators.NullOrContains;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
@FormClass( validators = {@Validator(FromTableForm.FormValidator.class)})
|
||||
public class FromTableForm {
|
||||
@FormClass( validators = {@Validator(FromJobConfig.FormValidator.class)})
|
||||
public class FromJobConfig {
|
||||
@Input(size = 50)
|
||||
public String schemaName;
|
||||
|
||||
|
@ -51,14 +51,14 @@ public class FromTableForm {
|
|||
@Input(size = 50)
|
||||
public String boundaryQuery;
|
||||
|
||||
public static class FormValidator extends AbstractValidator<FromTableForm> {
|
||||
public static class FormValidator extends AbstractValidator<FromJobConfig> {
|
||||
@Override
|
||||
public void validate(FromTableForm form) {
|
||||
public void validate(FromJobConfig form) {
|
||||
if(form.tableName == null && form.sql == null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Either fromTable name or SQL must be specified");
|
||||
addMessage(Status.UNACCEPTABLE, "Either table name or SQL must be specified");
|
||||
}
|
||||
if(form.tableName != null && form.sql != null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Both fromTable name and SQL cannot be specified");
|
||||
addMessage(Status.UNACCEPTABLE, "Both table name and SQL cannot be specified");
|
||||
}
|
||||
if(form.schemaName != null && form.sql != null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Both schema name and SQL cannot be specified");
|
|
@ -25,9 +25,9 @@ import org.apache.sqoop.model.Form;
|
|||
*/
|
||||
@ConfigurationClass
|
||||
public class FromJobConfiguration {
|
||||
@Form public FromTableForm fromTable;
|
||||
@Form public FromJobConfig fromJobConfig;
|
||||
|
||||
public FromJobConfiguration() {
|
||||
fromTable = new FromTableForm();
|
||||
fromJobConfig = new FromJobConfig();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,8 +33,8 @@ import java.util.Map;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
@FormClass(validators = {@Validator(ConnectionForm.FormValidator.class)})
|
||||
public class ConnectionForm {
|
||||
@FormClass(validators = {@Validator(LinkConfig.FormValidator.class)})
|
||||
public class LinkConfig {
|
||||
@Input(size = 128, validators = {@Validator(NotEmpty.class), @Validator(ClassAvailable.class)} )
|
||||
public String jdbcDriver;
|
||||
|
||||
|
@ -50,12 +50,12 @@ public class ConnectionForm {
|
|||
@Input
|
||||
public Map<String, String> jdbcProperties;
|
||||
|
||||
public static class FormValidator extends AbstractValidator<ConnectionForm> {
|
||||
public static class FormValidator extends AbstractValidator<LinkConfig> {
|
||||
@Override
|
||||
public void validate(ConnectionForm form) {
|
||||
public void validate(LinkConfig linkConfig) {
|
||||
// See if we can connect to the database
|
||||
try {
|
||||
DriverManager.getConnection(form.connectionString, form.username, form.password);
|
||||
DriverManager.getConnection(linkConfig.connectionString, linkConfig.username, linkConfig.password);
|
||||
} catch (SQLException e) {
|
||||
addMessage(Status.ACCEPTABLE, "Can't connect to the database with given credentials: " + e.getMessage());
|
||||
}
|
|
@ -24,11 +24,11 @@ import org.apache.sqoop.model.Form;
|
|||
*
|
||||
*/
|
||||
@ConfigurationClass
|
||||
public class ConnectionConfiguration {
|
||||
public class LinkConfiguration {
|
||||
|
||||
@Form public ConnectionForm connection;
|
||||
@Form public LinkConfig link;
|
||||
|
||||
public ConnectionConfiguration() {
|
||||
connection = new ConnectionForm();
|
||||
public LinkConfiguration() {
|
||||
link = new LinkConfig();
|
||||
}
|
||||
}
|
|
@ -26,8 +26,8 @@ import org.apache.sqoop.validation.validators.AbstractValidator;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
@FormClass(validators = {@Validator(ToTableForm.FormValidator.class)})
|
||||
public class ToTableForm {
|
||||
@FormClass(validators = {@Validator(ToJobConfig.FormValidator.class)})
|
||||
public class ToJobConfig {
|
||||
@Input(size = 50) public String schemaName;
|
||||
@Input(size = 2000) public String tableName;
|
||||
@Input(size = 50) public String sql;
|
||||
|
@ -35,20 +35,20 @@ public class ToTableForm {
|
|||
@Input(size = 2000) public String stageTableName;
|
||||
@Input public Boolean clearStageTable;
|
||||
|
||||
public static class FormValidator extends AbstractValidator<ToTableForm> {
|
||||
public static class FormValidator extends AbstractValidator<ToJobConfig> {
|
||||
@Override
|
||||
public void validate(ToTableForm form) {
|
||||
public void validate(ToJobConfig form) {
|
||||
if(form.tableName == null && form.sql == null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Either fromTable name or SQL must be specified");
|
||||
addMessage(Status.UNACCEPTABLE, "Either table name or SQL must be specified");
|
||||
}
|
||||
if(form.tableName != null && form.sql != null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Both fromTable name and SQL cannot be specified");
|
||||
addMessage(Status.UNACCEPTABLE, "Both table name and SQL cannot be specified");
|
||||
}
|
||||
if(form.tableName == null && form.stageTableName != null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Stage fromTable name cannot be specified without specifying fromTable name");
|
||||
addMessage(Status.UNACCEPTABLE, "Stage table name cannot be specified without specifying table name");
|
||||
}
|
||||
if(form.stageTableName == null && form.clearStageTable != null) {
|
||||
addMessage(Status.UNACCEPTABLE, "Clear stage fromTable cannot be specified without specifying name of the stage fromTable.");
|
||||
addMessage(Status.UNACCEPTABLE, "Clear stage table cannot be specified without specifying name of the stage table.");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,9 +25,9 @@ import org.apache.sqoop.model.Form;
|
|||
*/
|
||||
@ConfigurationClass
|
||||
public class ToJobConfiguration {
|
||||
@Form public ToTableForm toTable;
|
||||
@Form public ToJobConfig toJobConfig;
|
||||
|
||||
public ToJobConfiguration() {
|
||||
toTable = new ToTableForm();
|
||||
toJobConfig = new ToJobConfig();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,127 @@
|
|||
# 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
|
||||
|
||||
############################
|
||||
# Link Config
|
||||
#
|
||||
link.label = Link configuration
|
||||
link.help = You must supply the information requested in order to \
|
||||
create a connection object.
|
||||
|
||||
# jdbc driver
|
||||
link.jdbcDriver.label = JDBC Driver Class
|
||||
link.jdbcDriver.help = Enter the fully qualified class name of the JDBC \
|
||||
driver that will be used for establishing this connection.
|
||||
|
||||
# connect string
|
||||
link.connectionString.label = JDBC Connection String
|
||||
link.connectionString.help = Enter the value of JDBC connection string to be \
|
||||
used by this connector for creating connections.
|
||||
|
||||
# username string
|
||||
link.username.label = Username
|
||||
link.username.help = Enter the username to be used for connecting to the \
|
||||
database.
|
||||
|
||||
# password string
|
||||
link.password.label = Password
|
||||
link.password.help = Enter the password to be used for connecting to the \
|
||||
database.
|
||||
|
||||
# jdbc properties
|
||||
link.jdbcProperties.label = JDBC Connection Properties
|
||||
link.jdbcProperties.help = Enter any JDBC properties that should be \
|
||||
supplied during the creation of connection.
|
||||
|
||||
# From Job Config
|
||||
#
|
||||
fromJobConfig.label = From database configuration
|
||||
fromJobConfig.help = You must supply the information requested in order to create \
|
||||
a job object.
|
||||
|
||||
# From schema name
|
||||
fromJobConfig.schemaName.label = Schema name
|
||||
fromJobConfig.schemaName.help = Schema name to process data in the remote database
|
||||
|
||||
# From table name
|
||||
fromJobConfig.tableName.label = Table name
|
||||
fromJobConfig.tableName.help = Table name to process data in the remote database
|
||||
|
||||
# From table SQL
|
||||
fromJobConfig.sql.label = Table SQL statement
|
||||
fromJobConfig.sql.help = SQL statement to process data in the remote database
|
||||
|
||||
# From table columns
|
||||
fromJobConfig.columns.label = Table column names
|
||||
fromJobConfig.columns.help = Specific columns of a table name or a table SQL
|
||||
|
||||
# From table warehouse
|
||||
fromJobConfig.warehouse.label = Data warehouse
|
||||
fromJobConfig.warehouse.help = The root directory for data
|
||||
|
||||
# From table datadir
|
||||
fromJobConfig.dataDirectory.label = Data directory
|
||||
fromJobConfig.dataDirectory.help = The sub-directory under warehouse for data
|
||||
|
||||
# From table pcol
|
||||
fromJobConfig.partitionColumn.label = Partition column name
|
||||
fromJobConfig.partitionColumn.help = A specific column for data partition
|
||||
|
||||
# From table pcol is null
|
||||
fromJobConfig.partitionColumnNull.label = Nulls in partition column
|
||||
fromJobConfig.partitionColumnNull.help = Whether there are null values in partition column
|
||||
|
||||
# From table boundary
|
||||
fromJobConfig.boundaryQuery.label = Boundary query
|
||||
fromJobConfig.boundaryQuery.help = The boundary query for data partition
|
||||
|
||||
# ToJob Config
|
||||
#
|
||||
toJobConfig.label = To database configuration
|
||||
toJobConfig.help = You must supply the information requested in order to create \
|
||||
a job object.
|
||||
|
||||
# To schema name
|
||||
toJobConfig.schemaName.label = Schema name
|
||||
toJobConfig.schemaName.help = Schema name to process data in the remote database
|
||||
|
||||
# To table name
|
||||
toJobConfig.tableName.label = Table name
|
||||
toJobConfig.tableName.help = Table name to process data in the remote database
|
||||
|
||||
# To table SQL
|
||||
toJobConfig.sql.label = Table SQL statement
|
||||
toJobConfig.sql.help = SQL statement to process data in the remote database
|
||||
|
||||
# To table columns
|
||||
toJobConfig.columns.label = Table column names
|
||||
toJobConfig.columns.help = Specific columns of a table name or a table SQL
|
||||
|
||||
# To stage table name
|
||||
toJobConfig.stageTableName.label = Stage table name
|
||||
toJobConfig.stageTableName.help = Name of the stage table to use
|
||||
|
||||
# To clear stage table
|
||||
toJobConfig.clearStageTable.label = Clear stage table
|
||||
toJobConfig.clearStageTable.help = Indicate if the stage table should be cleared
|
||||
|
||||
# Placeholders to have some entities created
|
||||
ignored.label = Ignored
|
||||
ignored.help = This is completely ignored
|
||||
|
||||
ignored.ignored.label = Ignored
|
||||
ignored.ignored.help = This is completely ignored
|
|
@ -1,127 +0,0 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
# Generic JDBC Connector Resources
|
||||
|
||||
############################
|
||||
# Connection Form
|
||||
#
|
||||
connection.label = Connection configuration
|
||||
connection.help = You must supply the information requested in order to \
|
||||
create a connection object.
|
||||
|
||||
# jdbc driver
|
||||
connection.jdbcDriver.label = JDBC Driver Class
|
||||
connection.jdbcDriver.help = Enter the fully qualified class name of the JDBC \
|
||||
driver that will be used for establishing this connection.
|
||||
|
||||
# connect string
|
||||
connection.connectionString.label = JDBC Connection String
|
||||
connection.connectionString.help = Enter the value of JDBC connection string to be \
|
||||
used by this connector for creating connections.
|
||||
|
||||
# username string
|
||||
connection.username.label = Username
|
||||
connection.username.help = Enter the username to be used for connecting to the \
|
||||
database.
|
||||
|
||||
# password string
|
||||
connection.password.label = Password
|
||||
connection.password.help = Enter the password to be used for connecting to the \
|
||||
database.
|
||||
|
||||
# jdbc properties
|
||||
connection.jdbcProperties.label = JDBC Connection Properties
|
||||
connection.jdbcProperties.help = Enter any JDBC properties that should be \
|
||||
supplied during the creation of connection.
|
||||
|
||||
# From Table Form
|
||||
#
|
||||
fromTable.label = From database configuration
|
||||
fromTable.help = You must supply the information requested in order to create \
|
||||
a job object.
|
||||
|
||||
# From schema name
|
||||
fromTable.schemaName.label = Schema name
|
||||
fromTable.schemaName.help = Schema name to process data in the remote database
|
||||
|
||||
# From table name
|
||||
fromTable.tableName.label = Table name
|
||||
fromTable.tableName.help = Table name to process data in the remote database
|
||||
|
||||
# From table SQL
|
||||
fromTable.sql.label = Table SQL statement
|
||||
fromTable.sql.help = SQL statement to process data in the remote database
|
||||
|
||||
# From table columns
|
||||
fromTable.columns.label = Table column names
|
||||
fromTable.columns.help = Specific columns of a table name or a table SQL
|
||||
|
||||
# From table warehouse
|
||||
fromTable.warehouse.label = Data warehouse
|
||||
fromTable.warehouse.help = The root directory for data
|
||||
|
||||
# From table datadir
|
||||
fromTable.dataDirectory.label = Data directory
|
||||
fromTable.dataDirectory.help = The sub-directory under warehouse for data
|
||||
|
||||
# From table pcol
|
||||
fromTable.partitionColumn.label = Partition column name
|
||||
fromTable.partitionColumn.help = A specific column for data partition
|
||||
|
||||
# From table pcol is null
|
||||
fromTable.partitionColumnNull.label = Nulls in partition column
|
||||
fromTable.partitionColumnNull.help = Whether there are null values in partition column
|
||||
|
||||
# From table boundary
|
||||
fromTable.boundaryQuery.label = Boundary query
|
||||
fromTable.boundaryQuery.help = The boundary query for data partition
|
||||
|
||||
# To table form
|
||||
#
|
||||
toTable.label = To database configuration
|
||||
toTable.help = You must supply the information requested in order to create \
|
||||
a job object.
|
||||
|
||||
# From schema name
|
||||
toTable.schemaName.label = Schema name
|
||||
toTable.schemaName.help = Schema name to process data in the remote database
|
||||
|
||||
# From table name
|
||||
toTable.tableName.label = Table name
|
||||
toTable.tableName.help = Table name to process data in the remote database
|
||||
|
||||
# From table SQL
|
||||
toTable.sql.label = Table SQL statement
|
||||
toTable.sql.help = SQL statement to process data in the remote database
|
||||
|
||||
# From table columns
|
||||
toTable.columns.label = Table column names
|
||||
toTable.columns.help = Specific columns of a table name or a table SQL
|
||||
|
||||
# To stage table name
|
||||
toTable.stageTableName.label = Stage table name
|
||||
toTable.stageTableName.help = Name of the stage table to use
|
||||
|
||||
# To clear stage table
|
||||
toTable.clearStageTable.label = Clear stage table
|
||||
toTable.clearStageTable.help = Indicate if the stage table should be cleared
|
||||
|
||||
# Placeholders to have some entities created
|
||||
ignored.label = Ignored
|
||||
ignored.help = This is completely ignored
|
||||
|
||||
ignored.ignored.label = Ignored
|
||||
ignored.ignored.help = This is completely ignored
|
|
@ -21,7 +21,7 @@ import junit.framework.TestCase;
|
|||
|
||||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.MutableMapContext;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Extractor;
|
||||
import org.apache.sqoop.job.etl.ExtractorContext;
|
||||
|
@ -67,10 +67,10 @@ public class TestExtractor extends TestCase {
|
|||
public void testQuery() throws Exception {
|
||||
MutableContext context = new MutableMapContext();
|
||||
|
||||
ConnectionConfiguration connectionConfig = new ConnectionConfiguration();
|
||||
LinkConfiguration connectionConfig = new LinkConfiguration();
|
||||
|
||||
connectionConfig.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connectionConfig.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
connectionConfig.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connectionConfig.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
|
||||
FromJobConfiguration jobConfig = new FromJobConfiguration();
|
||||
|
||||
|
@ -99,10 +99,10 @@ public class TestExtractor extends TestCase {
|
|||
public void testSubquery() throws Exception {
|
||||
MutableContext context = new MutableMapContext();
|
||||
|
||||
ConnectionConfiguration connectionConfig = new ConnectionConfiguration();
|
||||
LinkConfiguration connectionConfig = new LinkConfiguration();
|
||||
|
||||
connectionConfig.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connectionConfig.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
connectionConfig.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connectionConfig.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
|
||||
FromJobConfiguration jobConfig = new FromJobConfiguration();
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ import junit.framework.TestCase;
|
|||
|
||||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.MutableMapContext;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.Constants;
|
||||
import org.apache.sqoop.job.etl.Initializer;
|
||||
|
@ -112,12 +112,12 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableName() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.tableName = schemalessTableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.tableName = schemalessTableName;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -138,13 +138,13 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableNameWithTableColumns() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.tableName = schemalessTableName;
|
||||
jobConf.fromTable.columns = tableColumns;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.tableName = schemalessTableName;
|
||||
jobConf.fromJobConfig.columns = tableColumns;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -165,13 +165,13 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableSql() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.sql = schemalessTableSql;
|
||||
jobConf.fromTable.partitionColumn = "DCOL";
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.sql = schemalessTableSql;
|
||||
jobConf.fromJobConfig.partitionColumn = "DCOL";
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -192,14 +192,14 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableSqlWithTableColumns() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.sql = schemalessTableSql;
|
||||
jobConf.fromTable.columns = tableColumns;
|
||||
jobConf.fromTable.partitionColumn = "DCOL";
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.sql = schemalessTableSql;
|
||||
jobConf.fromJobConfig.columns = tableColumns;
|
||||
jobConf.fromJobConfig.partitionColumn = "DCOL";
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -221,15 +221,15 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableNameWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.schemaName = schemaName;
|
||||
jobConf.fromTable.tableName = tableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.schemaName = schemaName;
|
||||
jobConf.fromJobConfig.tableName = tableName;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -250,16 +250,16 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableNameWithTableColumnsWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.schemaName = schemaName;
|
||||
jobConf.fromTable.tableName = tableName;
|
||||
jobConf.fromTable.columns = tableColumns;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.schemaName = schemaName;
|
||||
jobConf.fromJobConfig.tableName = tableName;
|
||||
jobConf.fromJobConfig.columns = tableColumns;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -280,16 +280,16 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableSqlWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.schemaName = schemaName;
|
||||
jobConf.fromTable.sql = tableSql;
|
||||
jobConf.fromTable.partitionColumn = "DCOL";
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.schemaName = schemaName;
|
||||
jobConf.fromJobConfig.sql = tableSql;
|
||||
jobConf.fromJobConfig.partitionColumn = "DCOL";
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -311,14 +311,14 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testGetSchemaForTable() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.schemaName = schemaName;
|
||||
jobConf.fromTable.tableName = tableName;
|
||||
jobConf.fromTable.partitionColumn = "DCOL";
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.schemaName = schemaName;
|
||||
jobConf.fromJobConfig.tableName = tableName;
|
||||
jobConf.fromJobConfig.partitionColumn = "DCOL";
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -327,19 +327,19 @@ public class TestFromInitializer extends TestCase {
|
|||
Initializer initializer = new GenericJdbcFromInitializer();
|
||||
initializer.initialize(initializerContext, connConf, jobConf);
|
||||
Schema schema = initializer.getSchema(initializerContext, connConf, jobConf);
|
||||
assertEquals(getSchema(jobConf.fromTable.schemaName + "." + tableName), schema);
|
||||
assertEquals(getSchema(jobConf.fromJobConfig.schemaName + "." + tableName), schema);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testGetSchemaForSql() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.schemaName = schemaName;
|
||||
jobConf.fromTable.sql = tableSql;
|
||||
jobConf.fromTable.partitionColumn = "DCOL";
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.schemaName = schemaName;
|
||||
jobConf.fromJobConfig.sql = tableSql;
|
||||
jobConf.fromJobConfig.partitionColumn = "DCOL";
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -353,17 +353,17 @@ public class TestFromInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableSqlWithTableColumnsWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromTable.schemaName = schemaName;
|
||||
jobConf.fromTable.sql = tableSql;
|
||||
jobConf.fromTable.columns = tableColumns;
|
||||
jobConf.fromTable.partitionColumn = "DCOL";
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.fromJobConfig.schemaName = schemaName;
|
||||
jobConf.fromJobConfig.sql = tableSql;
|
||||
jobConf.fromJobConfig.columns = tableColumns;
|
||||
jobConf.fromJobConfig.partitionColumn = "DCOL";
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Collection;
|
|||
|
||||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.MutableMapContext;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.etl.io.DataReader;
|
||||
import org.apache.sqoop.job.etl.Loader;
|
||||
|
@ -82,10 +82,10 @@ public class TestLoader {
|
|||
public void testInsert() throws Exception {
|
||||
MutableContext context = new MutableMapContext();
|
||||
|
||||
ConnectionConfiguration connectionConfig = new ConnectionConfiguration();
|
||||
LinkConfiguration connectionConfig = new LinkConfiguration();
|
||||
|
||||
connectionConfig.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connectionConfig.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
connectionConfig.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connectionConfig.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
|
||||
ToJobConfiguration jobConfig = new ToJobConfiguration();
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import junit.framework.TestCase;
|
|||
|
||||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.MutableMapContext;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Partition;
|
||||
import org.apache.sqoop.job.etl.Partitioner;
|
||||
|
@ -55,7 +55,7 @@ public class TestPartitioner extends TestCase {
|
|||
GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
|
||||
String.valueOf(START + NUMBER_OF_ROWS - 1));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -86,7 +86,7 @@ public class TestPartitioner extends TestCase {
|
|||
GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
|
||||
String.valueOf(START + NUMBER_OF_ROWS - 1));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -115,7 +115,7 @@ public class TestPartitioner extends TestCase {
|
|||
GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
|
||||
String.valueOf(START + NUMBER_OF_ROWS - 1));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -151,7 +151,7 @@ public class TestPartitioner extends TestCase {
|
|||
GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
|
||||
String.valueOf((double)(START + NUMBER_OF_ROWS - 1)));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -182,7 +182,7 @@ public class TestPartitioner extends TestCase {
|
|||
GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
|
||||
String.valueOf((double)(START + NUMBER_OF_ROWS - 1)));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -203,7 +203,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE, String.valueOf(START));
|
||||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE, String.valueOf(START + NUMBER_OF_ROWS - 1));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -226,7 +226,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE, String.valueOf(new BigDecimal(START)));
|
||||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE, String.valueOf(new BigDecimal(START + NUMBER_OF_ROWS - 1)));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -247,7 +247,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE, String.valueOf(new BigDecimal(START)));
|
||||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE, String.valueOf(new BigDecimal(START)));
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -272,7 +272,7 @@ public class TestPartitioner extends TestCase {
|
|||
.toString());
|
||||
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -300,7 +300,7 @@ public class TestPartitioner extends TestCase {
|
|||
Time.valueOf("10:40:50").toString());
|
||||
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -325,7 +325,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
|
||||
Timestamp.valueOf("2013-12-31 10:40:50.654").toString());
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -349,7 +349,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants
|
||||
.CONNECTOR_JDBC_PARTITION_MAXVALUE, "1");
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -372,7 +372,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants
|
||||
.CONNECTOR_JDBC_PARTITION_MAXVALUE, "Z");
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -419,7 +419,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants
|
||||
.CONNECTOR_JDBC_PARTITION_MAXVALUE, "Warty Warthog");
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
PartitionerContext partitionerContext = new PartitionerContext(context, 5, null);
|
||||
|
@ -442,7 +442,7 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants
|
||||
.CONNECTOR_JDBC_PARTITION_MAXVALUE, "AAF");
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
|
@ -471,9 +471,9 @@ public class TestPartitioner extends TestCase {
|
|||
context.setString(GenericJdbcConnectorConstants
|
||||
.CONNECTOR_JDBC_PARTITION_MAXVALUE, "AAE");
|
||||
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
jobConf.fromTable.partitionColumnNull = true;
|
||||
jobConf.fromJobConfig.partitionColumnNull = true;
|
||||
|
||||
Partitioner partitioner = new GenericJdbcPartitioner();
|
||||
PartitionerContext partitionerContext = new PartitionerContext(context, 5, null);
|
||||
|
|
|
@ -21,7 +21,7 @@ import junit.framework.TestCase;
|
|||
import org.apache.sqoop.common.MutableContext;
|
||||
import org.apache.sqoop.common.MutableMapContext;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Initializer;
|
||||
import org.apache.sqoop.job.etl.InitializerContext;
|
||||
|
@ -75,14 +75,14 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableName() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemalessTableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -96,15 +96,15 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableNameWithTableColumns() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemalessTableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
jobConf.toTable.columns = tableColumns;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
jobConf.toJobConfig.columns = tableColumns;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -118,12 +118,12 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableSql() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.sql = schemalessTableSql;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.sql = schemalessTableSql;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -137,15 +137,15 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableNameWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.schemaName = schemaName;
|
||||
jobConf.toTable.tableName = tableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.schemaName = schemaName;
|
||||
jobConf.toJobConfig.tableName = tableName;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -159,16 +159,16 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableNameWithTableColumnsWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.schemaName = schemaName;
|
||||
jobConf.toTable.tableName = tableName;
|
||||
jobConf.toTable.columns = tableColumns;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.schemaName = schemaName;
|
||||
jobConf.toJobConfig.tableName = tableName;
|
||||
jobConf.toJobConfig.columns = tableColumns;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -182,13 +182,13 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTableSqlWithSchema() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.schemaName = schemaName;
|
||||
jobConf.toTable.sql = tableSql;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.schemaName = schemaName;
|
||||
jobConf.toJobConfig.sql = tableSql;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -216,13 +216,13 @@ public class TestToInitializer extends TestCase {
|
|||
}
|
||||
|
||||
public void testNonExistingStageTable() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
jobConf.toTable.stageTableName = stageTableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
jobConf.toJobConfig.stageTableName = stageTableName;
|
||||
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
@ -239,15 +239,15 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testNonEmptyStageTable() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullStageTableName = executor.delimitIdentifier(stageTableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
jobConf.toTable.stageTableName = stageTableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
jobConf.toJobConfig.stageTableName = stageTableName;
|
||||
createTable(fullStageTableName);
|
||||
executor.executeUpdate("INSERT INTO " + fullStageTableName +
|
||||
" VALUES(1, 1.1, 'one')");
|
||||
|
@ -264,17 +264,16 @@ public class TestToInitializer extends TestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testClearStageTableValidation() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
//specifying clear stage table flag without specifying name of
|
||||
// the stage table
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
jobConf.toTable.clearStageTable = false;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
jobConf.toJobConfig.clearStageTable = false;
|
||||
ValidationRunner validationRunner = new ValidationRunner();
|
||||
ValidationResult result = validationRunner.validate(jobConf);
|
||||
assertEquals("User should not specify clear stage table flag without " +
|
||||
|
@ -282,49 +281,48 @@ public class TestToInitializer extends TestCase {
|
|||
Status.UNACCEPTABLE,
|
||||
result.getStatus());
|
||||
assertTrue(result.getMessages().containsKey(
|
||||
"toTable"));
|
||||
"toJobConfig"));
|
||||
|
||||
jobConf.toTable.clearStageTable = true;
|
||||
jobConf.toJobConfig.clearStageTable = true;
|
||||
result = validationRunner.validate(jobConf);
|
||||
assertEquals("User should not specify clear stage table flag without " +
|
||||
"specifying name of the stage table",
|
||||
Status.UNACCEPTABLE,
|
||||
result.getStatus());
|
||||
assertTrue(result.getMessages().containsKey(
|
||||
"toTable"));
|
||||
"toJobConfig"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testStageTableWithoutTable() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
//specifying stage table without specifying table name
|
||||
jobConf.toTable.stageTableName = stageTableName;
|
||||
jobConf.toTable.sql = "";
|
||||
jobConf.toJobConfig.stageTableName = stageTableName;
|
||||
jobConf.toJobConfig.sql = "";
|
||||
|
||||
ValidationRunner validationRunner = new ValidationRunner();
|
||||
ValidationResult result = validationRunner.validate(jobConf);
|
||||
assertEquals("Stage table name cannot be specified without specifying " +
|
||||
"table name", Status.UNACCEPTABLE, result.getStatus());
|
||||
assertTrue(result.getMessages().containsKey(
|
||||
"toTable"));
|
||||
"toJobConfig"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testClearStageTable() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullStageTableName = executor.delimitIdentifier(stageTableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
jobConf.toTable.stageTableName = stageTableName;
|
||||
jobConf.toTable.clearStageTable = true;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
jobConf.toJobConfig.stageTableName = stageTableName;
|
||||
jobConf.toJobConfig.clearStageTable = true;
|
||||
createTable(fullStageTableName);
|
||||
executor.executeUpdate("INSERT INTO " + fullStageTableName +
|
||||
" VALUES(1, 1.1, 'one')");
|
||||
|
@ -340,15 +338,15 @@ public class TestToInitializer extends TestCase {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testStageTable() throws Exception {
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
|
||||
String fullStageTableName = executor.delimitIdentifier(stageTableName);
|
||||
|
||||
connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.connection.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toTable.tableName = schemalessTableName;
|
||||
jobConf.toTable.stageTableName = stageTableName;
|
||||
connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
|
||||
connConf.link.connectionString = GenericJdbcTestConstants.URL;
|
||||
jobConf.toJobConfig.tableName = schemalessTableName;
|
||||
jobConf.toJobConfig.stageTableName = stageTableName;
|
||||
createTable(fullStageTableName);
|
||||
MutableContext context = new MutableMapContext();
|
||||
InitializerContext initializerContext = new InitializerContext(context);
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.sqoop.connector.hdfs;
|
|||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||
import org.apache.sqoop.connector.spi.RepositoryUpgrader;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.apache.sqoop.model.MInput;
|
||||
|
@ -30,9 +30,9 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class HdfsMetadataUpgrader extends MetadataUpgrader {
|
||||
public class HdfsConfigUpgrader extends RepositoryUpgrader {
|
||||
private static final Logger LOG =
|
||||
Logger.getLogger(HdfsMetadataUpgrader.class);
|
||||
Logger.getLogger(HdfsConfigUpgrader.class);
|
||||
|
||||
/*
|
||||
* For now, there is no real upgrade. So copy all data over,
|
|
@ -20,10 +20,10 @@ package org.apache.sqoop.connector.hdfs;
|
|||
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.common.VersionInfo;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||
import org.apache.sqoop.connector.spi.RepositoryUpgrader;
|
||||
import org.apache.sqoop.connector.spi.SqoopConnector;
|
||||
import org.apache.sqoop.job.etl.From;
|
||||
import org.apache.sqoop.job.etl.To;
|
||||
|
@ -71,8 +71,8 @@ public class HdfsConnector extends SqoopConnector {
|
|||
* @return Get connection configuration class
|
||||
*/
|
||||
@Override
|
||||
public Class getConnectionConfigurationClass() {
|
||||
return ConnectionConfiguration.class;
|
||||
public Class getLinkConfigurationClass() {
|
||||
return LinkConfiguration.class;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -108,7 +108,7 @@ public class HdfsConnector extends SqoopConnector {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns validation object that Sqoop framework can use to validate user
|
||||
* Returns validation object that Sqoop can use to validate user
|
||||
* supplied forms before accepting them. This object will be used both for
|
||||
* connection and job forms.
|
||||
*
|
||||
|
@ -120,13 +120,13 @@ public class HdfsConnector extends SqoopConnector {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns an {@linkplain org.apache.sqoop.connector.spi.MetadataUpgrader} object that can upgrade the
|
||||
* Returns an {@linkplain org.apache.sqoop.connector.spi.RepositoryUpgrader} object that can upgrade the
|
||||
* connection and job metadata.
|
||||
*
|
||||
* @return MetadataUpgrader object
|
||||
*/
|
||||
@Override
|
||||
public MetadataUpgrader getMetadataUpgrader() {
|
||||
return new HdfsMetadataUpgrader();
|
||||
public RepositoryUpgrader getRepositoryUpgrader() {
|
||||
return new HdfsConfigUpgrader();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ public final class HdfsConstants extends Constants {
|
|||
|
||||
// Resource bundle name
|
||||
public static final String RESOURCE_BUNDLE_NAME =
|
||||
"hdfs-connector-resources";
|
||||
"hdfs-connector-config";
|
||||
|
||||
public static final char DEFAULT_RECORD_DELIMITER = '\n';
|
||||
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.hadoop.io.compress.CompressionCodec;
|
|||
import org.apache.hadoop.io.compress.CompressionCodecFactory;
|
||||
import org.apache.hadoop.util.LineReader;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.etl.io.DataWriter;
|
||||
import org.apache.sqoop.job.etl.Extractor;
|
||||
|
@ -42,7 +42,7 @@ import java.io.IOException;
|
|||
* Extract from HDFS.
|
||||
* Default field delimiter of a record is comma.
|
||||
*/
|
||||
public class HdfsExtractor extends Extractor<ConnectionConfiguration, FromJobConfiguration, HdfsPartition> {
|
||||
public class HdfsExtractor extends Extractor<LinkConfiguration, FromJobConfiguration, HdfsPartition> {
|
||||
|
||||
public static final Logger LOG = Logger.getLogger(HdfsExtractor.class);
|
||||
|
||||
|
@ -52,7 +52,7 @@ public class HdfsExtractor extends Extractor<ConnectionConfiguration, FromJobCon
|
|||
|
||||
@Override
|
||||
public void extract(ExtractorContext context,
|
||||
ConnectionConfiguration connectionConfiguration,
|
||||
LinkConfiguration connectionConfiguration,
|
||||
FromJobConfiguration jobConfiguration, HdfsPartition partition) {
|
||||
|
||||
conf = ((PrefixContext) context.getContext()).getConfiguration();
|
||||
|
|
|
@ -29,17 +29,16 @@ public class HdfsInitializer extends Initializer {
|
|||
* promoted to all other part of the workflow automatically.
|
||||
*
|
||||
* @param context Initializer context object
|
||||
* @param connection Connector's connection configuration object
|
||||
* @param job Connector's job configuration object
|
||||
* @param linkConf Connector's link configuration object
|
||||
* @param jobConf Connector's job configuration object
|
||||
*/
|
||||
@Override
|
||||
public void initialize(InitializerContext context, Object connection, Object job) {
|
||||
public void initialize(InitializerContext context, Object linkConf, Object jobConf) {
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Schema getSchema(InitializerContext context, Object connection, Object job) {
|
||||
public Schema getSchema(InitializerContext context, Object linkConf, Object jobConf) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,8 +23,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.sqoop.common.PrefixContext;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.OutputFormat;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.hdfsWriter.GenericHdfsWriter;
|
||||
import org.apache.sqoop.connector.hdfs.hdfsWriter.HdfsSequenceWriter;
|
||||
|
@ -37,24 +37,24 @@ import org.apache.sqoop.utils.ClassUtils;
|
|||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
|
||||
public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfiguration> {
|
||||
public class HdfsLoader extends Loader<LinkConfiguration, ToJobConfiguration> {
|
||||
/**
|
||||
* Load data to target.
|
||||
*
|
||||
* @param context Loader context object
|
||||
* @param connection Connection configuration
|
||||
* @param job Job configuration
|
||||
* @param linkConf Link configuration
|
||||
* @param toJobConf Job configuration
|
||||
* @throws Exception
|
||||
*/
|
||||
@Override
|
||||
public void load(LoaderContext context, ConnectionConfiguration connection, ToJobConfiguration job) throws Exception {
|
||||
public void load(LoaderContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) throws Exception {
|
||||
|
||||
DataReader reader = context.getDataReader();
|
||||
|
||||
Configuration conf = ((PrefixContext)context.getContext()).getConfiguration();
|
||||
|
||||
String directoryName = job.output.outputDirectory;
|
||||
String codecname = getCompressionCodecName(job);
|
||||
String directoryName = toJobConf.toJobConfig.outputDirectory;
|
||||
String codecname = getCompressionCodecName(toJobConf);
|
||||
|
||||
CompressionCodec codec = null;
|
||||
if (codecname != null) {
|
||||
|
@ -73,12 +73,12 @@ public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfigurati
|
|||
}
|
||||
}
|
||||
|
||||
String filename = directoryName + "/" + UUID.randomUUID() + getExtension(job,codec);
|
||||
String filename = directoryName + "/" + UUID.randomUUID() + getExtension(toJobConf,codec);
|
||||
|
||||
try {
|
||||
Path filepath = new Path(filename);
|
||||
|
||||
GenericHdfsWriter filewriter = getWriter(job);
|
||||
GenericHdfsWriter filewriter = getWriter(toJobConf);
|
||||
|
||||
filewriter.initialize(filepath,conf,codec);
|
||||
|
||||
|
@ -95,18 +95,15 @@ public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfigurati
|
|||
|
||||
}
|
||||
|
||||
private GenericHdfsWriter getWriter(ToJobConfiguration job) {
|
||||
if (job.output.outputFormat == OutputFormat.SEQUENCE_FILE)
|
||||
return new HdfsSequenceWriter();
|
||||
else
|
||||
return new HdfsTextWriter();
|
||||
private GenericHdfsWriter getWriter(ToJobConfiguration toJobConf) {
|
||||
return (toJobConf.toJobConfig.outputFormat == ToFormat.SEQUENCE_FILE) ? new HdfsSequenceWriter()
|
||||
: new HdfsTextWriter();
|
||||
}
|
||||
|
||||
|
||||
private String getCompressionCodecName(ToJobConfiguration jobConf) {
|
||||
if(jobConf.output.compression == null)
|
||||
private String getCompressionCodecName(ToJobConfiguration toJobConf) {
|
||||
if(toJobConf.toJobConfig.compression == null)
|
||||
return null;
|
||||
switch(jobConf.output.compression) {
|
||||
switch(toJobConf.toJobConfig.compression) {
|
||||
case NONE:
|
||||
return null;
|
||||
case DEFAULT:
|
||||
|
@ -124,14 +121,14 @@ public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfigurati
|
|||
case SNAPPY:
|
||||
return "org.apache.hadoop.io.compress.SnappyCodec";
|
||||
case CUSTOM:
|
||||
return jobConf.output.customCompression.trim();
|
||||
return toJobConf.toJobConfig.customCompression.trim();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
//TODO: We should probably support configurable extensions at some point
|
||||
private static String getExtension(ToJobConfiguration job, CompressionCodec codec) {
|
||||
if (job.output.outputFormat == OutputFormat.SEQUENCE_FILE)
|
||||
private static String getExtension(ToJobConfiguration toJobConf, CompressionCodec codec) {
|
||||
if (toJobConf.toJobConfig.outputFormat == ToFormat.SEQUENCE_FILE)
|
||||
return ".seq";
|
||||
if (codec == null)
|
||||
return ".txt";
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
|
|||
import org.apache.hadoop.net.NodeBase;
|
||||
import org.apache.hadoop.net.NetworkTopology;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Partition;
|
||||
import org.apache.sqoop.job.etl.Partitioner;
|
||||
|
@ -50,7 +50,7 @@ import org.apache.sqoop.common.PrefixContext;
|
|||
* This class derives mostly from CombineFileInputFormat of Hadoop, i.e.
|
||||
* org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.
|
||||
*/
|
||||
public class HdfsPartitioner extends Partitioner<ConnectionConfiguration, FromJobConfiguration> {
|
||||
public class HdfsPartitioner extends Partitioner<LinkConfiguration, FromJobConfiguration> {
|
||||
|
||||
public static final String SPLIT_MINSIZE_PERNODE =
|
||||
"mapreduce.input.fileinputformat.split.minsize.per.node";
|
||||
|
@ -68,12 +68,12 @@ public class HdfsPartitioner extends Partitioner<ConnectionConfiguration, FromJo
|
|||
|
||||
@Override
|
||||
public List<Partition> getPartitions(PartitionerContext context,
|
||||
ConnectionConfiguration connectionConfiguration, FromJobConfiguration jobConfiguration) {
|
||||
LinkConfiguration linkConfiguration, FromJobConfiguration jobConfiguration) {
|
||||
|
||||
Configuration conf = ((PrefixContext)context.getContext()).getConfiguration();
|
||||
|
||||
try {
|
||||
long numInputBytes = getInputSize(conf, jobConfiguration.input.inputDirectory);
|
||||
long numInputBytes = getInputSize(conf, jobConfiguration.fromJobConfig.inputDirectory);
|
||||
maxSplitSize = numInputBytes / context.getMaxPartitions();
|
||||
|
||||
if(numInputBytes % context.getMaxPartitions() != 0 ) {
|
||||
|
@ -118,7 +118,7 @@ public class HdfsPartitioner extends Partitioner<ConnectionConfiguration, FromJo
|
|||
}
|
||||
|
||||
// all the files in input set
|
||||
String indir = jobConfiguration.input.inputDirectory;
|
||||
String indir = jobConfiguration.fromJobConfig.inputDirectory;
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
|
||||
List<Path> paths = new LinkedList<Path>();
|
||||
|
@ -147,7 +147,7 @@ public class HdfsPartitioner extends Partitioner<ConnectionConfiguration, FromJo
|
|||
}
|
||||
}
|
||||
|
||||
//TODO: Perhaps get the FS from connection configuration so we can support remote HDFS
|
||||
//TODO: Perhaps get the FS from link configuration so we can support remote HDFS
|
||||
private long getInputSize(Configuration conf, String indir) throws IOException {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
FileStatus[] files = fs.listStatus(new Path(indir));
|
||||
|
|
|
@ -23,13 +23,13 @@ import org.apache.sqoop.validation.Validation;
|
|||
import org.apache.sqoop.validation.Validator;
|
||||
|
||||
/**
|
||||
* Validate framework configuration objects
|
||||
* Validate configuration objects
|
||||
*/
|
||||
public class HdfsValidator extends Validator {
|
||||
|
||||
@Override
|
||||
public Validation validateConnection(Object connectionConfiguration) {
|
||||
Validation validation = new Validation(ConnectionConfiguration.class);
|
||||
public Validation validateLink(Object connectionConfiguration) {
|
||||
Validation validation = new Validation(LinkConfiguration.class);
|
||||
// No validation on connection object
|
||||
return validation;
|
||||
}
|
||||
|
@ -41,42 +41,39 @@ public class HdfsValidator extends Validator {
|
|||
return super.validateJob(jobConfiguration);
|
||||
}
|
||||
|
||||
private Validation validateExportJob(Object jobConfiguration) {
|
||||
@SuppressWarnings("unused")
|
||||
private Validation validateFromJob(Object jobConfiguration) {
|
||||
Validation validation = new Validation(FromJobConfiguration.class);
|
||||
FromJobConfiguration configuration = (FromJobConfiguration)jobConfiguration;
|
||||
|
||||
validateInputForm(validation, configuration.input);
|
||||
|
||||
|
||||
validateInputForm(validation, configuration.fromJobConfig);
|
||||
return validation;
|
||||
}
|
||||
|
||||
private Validation validateImportJob(Object jobConfiguration) {
|
||||
@SuppressWarnings("unused")
|
||||
private Validation validateToJob(Object jobConfiguration) {
|
||||
Validation validation = new Validation(ToJobConfiguration.class);
|
||||
ToJobConfiguration configuration = (ToJobConfiguration)jobConfiguration;
|
||||
|
||||
validateOutputForm(validation, configuration.output);
|
||||
|
||||
validateOutputForm(validation, configuration.toJobConfig);
|
||||
return validation;
|
||||
}
|
||||
|
||||
private void validateInputForm(Validation validation, InputForm input) {
|
||||
private void validateInputForm(Validation validation, FromJobConfig input) {
|
||||
if(input.inputDirectory == null || input.inputDirectory.isEmpty()) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "input", "inputDirectory", "Input directory is empty");
|
||||
}
|
||||
}
|
||||
|
||||
private void validateOutputForm(Validation validation, OutputForm output) {
|
||||
private void validateOutputForm(Validation validation, ToJobConfig output) {
|
||||
if(output.outputDirectory == null || output.outputDirectory.isEmpty()) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "output", "outputDirectory", "Output directory is empty");
|
||||
}
|
||||
if(output.customCompression != null &&
|
||||
output.customCompression.trim().length() > 0 &&
|
||||
output.compression != OutputCompression.CUSTOM) {
|
||||
output.compression != ToCompression.CUSTOM) {
|
||||
validation.addMessage(Status.UNACCEPTABLE, "output", "compression",
|
||||
"custom compression should be blank as " + output.compression + " is being used.");
|
||||
}
|
||||
if(output.compression == OutputCompression.CUSTOM &&
|
||||
if(output.compression == ToCompression.CUSTOM &&
|
||||
(output.customCompression == null ||
|
||||
output.customCompression.trim().length() == 0)
|
||||
) {
|
||||
|
@ -84,6 +81,4 @@ public class HdfsValidator extends Validator {
|
|||
"custom compression is blank.");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.sqoop.model.Input;
|
|||
*
|
||||
*/
|
||||
@FormClass
|
||||
public class InputForm {
|
||||
public class FromJobConfig {
|
||||
|
||||
@Input(size = 255) public String inputDirectory;
|
||||
}
|
|
@ -22,11 +22,10 @@ import org.apache.sqoop.model.Form;
|
|||
|
||||
@ConfigurationClass
|
||||
public class FromJobConfiguration {
|
||||
@Form public InputForm input;
|
||||
|
||||
@Form public FromJobConfig fromJobConfig;
|
||||
|
||||
public FromJobConfiguration() {
|
||||
input = new InputForm();
|
||||
fromJobConfig = new FromJobConfig();
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.apache.sqoop.model.FormClass;
|
|||
import org.apache.sqoop.model.Input;
|
||||
|
||||
@FormClass
|
||||
public class ConnectionForm {
|
||||
public class LinkConfig {
|
||||
//Todo: Didn't find anything that belongs here...
|
||||
// Since empty forms don't work (DERBYREPO_0008:The form contains no input metadata), I'm putting a dummy form here
|
||||
|
|
@ -21,11 +21,11 @@ import org.apache.sqoop.model.ConfigurationClass;
|
|||
import org.apache.sqoop.model.Form;
|
||||
|
||||
@ConfigurationClass
|
||||
public class ConnectionConfiguration {
|
||||
public class LinkConfiguration {
|
||||
@Form
|
||||
public ConnectionForm connection;
|
||||
public LinkConfig link;
|
||||
|
||||
public ConnectionConfiguration() {
|
||||
connection = new ConnectionForm();
|
||||
public LinkConfiguration() {
|
||||
link = new LinkConfig();
|
||||
}
|
||||
}
|
|
@ -20,7 +20,7 @@ package org.apache.sqoop.connector.hdfs.configuration;
|
|||
/**
|
||||
* Supported compressions
|
||||
*/
|
||||
public enum OutputCompression {
|
||||
public enum ToCompression {
|
||||
NONE,
|
||||
DEFAULT,
|
||||
DEFLATE,
|
|
@ -20,7 +20,7 @@ package org.apache.sqoop.connector.hdfs.configuration;
|
|||
/**
|
||||
* Various supported formats on disk
|
||||
*/
|
||||
public enum OutputFormat {
|
||||
public enum ToFormat {
|
||||
/**
|
||||
* Comma separated text file
|
||||
*/
|
|
@ -24,11 +24,11 @@ import org.apache.sqoop.model.Input;
|
|||
*
|
||||
*/
|
||||
@FormClass
|
||||
public class OutputForm {
|
||||
public class ToJobConfig {
|
||||
|
||||
@Input public OutputFormat outputFormat;
|
||||
@Input public ToFormat outputFormat;
|
||||
|
||||
@Input public OutputCompression compression;
|
||||
@Input public ToCompression compression;
|
||||
|
||||
@Input(size = 255) public String customCompression;
|
||||
|
|
@ -23,9 +23,9 @@ import org.apache.sqoop.model.Form;
|
|||
@ConfigurationClass
|
||||
public class ToJobConfiguration {
|
||||
@Form
|
||||
public OutputForm output;
|
||||
public ToJobConfig toJobConfig;
|
||||
|
||||
public ToJobConfiguration() {
|
||||
output = new OutputForm();
|
||||
toJobConfig = new ToJobConfig();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.hadoop.io.SequenceFile;
|
|||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HdfsSequenceWriter extends GenericHdfsWriter {
|
||||
|
@ -32,6 +31,7 @@ public class HdfsSequenceWriter extends GenericHdfsWriter {
|
|||
private SequenceFile.Writer filewriter;
|
||||
private Text text;
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public void initialize(Path filepath, Configuration conf, CompressionCodec codec) throws IOException {
|
||||
if (codec != null) {
|
||||
filewriter = SequenceFile.createWriter(filepath.getFileSystem(conf),
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
# 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 HDFS Connector Resources
|
||||
|
||||
############################
|
||||
# Link Config
|
||||
#
|
||||
link.label = Link configuration
|
||||
link.help = You must supply the information requested in order to \
|
||||
create a connection object.
|
||||
|
||||
link.dummy.label = Dummy parameter needed to get HDFS connector to register
|
||||
link.dummy.help = You can write anything here. Doesn't matter.
|
||||
|
||||
# To Job Config
|
||||
#
|
||||
toJobConfig.label = ToJob configuration
|
||||
toJobConfig.help = You must supply the information requested in order to \
|
||||
get information where you want to store your data.
|
||||
|
||||
toJobConfig.storageType.label = Storage type
|
||||
toJobConfig.storageType.help = Target on Hadoop ecosystem where to store data
|
||||
|
||||
toJobConfig.outputFormat.label = Output format
|
||||
toJobConfig.outputFormat.help = Format in which data should be serialized
|
||||
|
||||
toJobConfig.compression.label = Compression format
|
||||
toJobConfig.compression.help = Compression that should be used for the data
|
||||
|
||||
toJobConfig.customCompression.label = Custom compression format
|
||||
toJobConfig.customCompression.help = Full class name of the custom compression
|
||||
|
||||
toJobConfig.outputDirectory.label = Output directory
|
||||
toJobConfig.outputDirectory.help = Output directory for final data
|
||||
|
||||
toJobConfig.ignored.label = Ignored
|
||||
toJobConfig.ignored.help = This value is ignored
|
||||
|
||||
# From Job Config
|
||||
#
|
||||
fromJobConfig.label = From Job configuration
|
||||
fromJobConfig.help = Specifies information required to get data from Hadoop ecosystem
|
||||
|
||||
fromJobConfig.inputDirectory.label = Input directory
|
||||
fromJobConfig.inputDirectory.help = Directory that should be exported
|
|
@ -1,58 +0,0 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the "License"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
# Generic HDFS Connector Resources
|
||||
|
||||
############################
|
||||
# Connection Form
|
||||
#
|
||||
connection.label = Connection configuration
|
||||
connection.help = You must supply the information requested in order to \
|
||||
create a connection object.
|
||||
|
||||
connection.dummy.label = Dummy parameter needed to get HDFS connector to register
|
||||
connection.dummy.help = You can write anything here. Doesn't matter.
|
||||
|
||||
# Output From
|
||||
#
|
||||
output.label = Output configuration
|
||||
output.help = You must supply the information requested in order to \
|
||||
get information where you want to store your data.
|
||||
|
||||
output.storageType.label = Storage type
|
||||
output.storageType.help = Target on Hadoop ecosystem where to store data
|
||||
|
||||
output.outputFormat.label = Output format
|
||||
output.outputFormat.help = Format in which data should be serialized
|
||||
|
||||
output.compression.label = Compression format
|
||||
output.compression.help = Compression that should be used for the data
|
||||
|
||||
output.customCompression.label = Custom compression format
|
||||
output.customCompression.help = Full class name of the custom compression
|
||||
|
||||
output.outputDirectory.label = Output directory
|
||||
output.outputDirectory.help = Output directory for final data
|
||||
|
||||
output.ignored.label = Ignored
|
||||
output.ignored.help = This value is ignored
|
||||
|
||||
# Input Form
|
||||
#
|
||||
input.label = Input configuration
|
||||
input.help = Specifies information required to get data from Hadoop ecosystem
|
||||
|
||||
input.inputDirectory.label = Input directory
|
||||
input.inputDirectory.help = Directory that should be exported
|
|
@ -22,9 +22,9 @@ import org.apache.hadoop.io.compress.BZip2Codec;
|
|||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.io.compress.DefaultCodec;
|
||||
import org.apache.sqoop.common.PrefixContext;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.OutputFormat;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
|
||||
import org.apache.sqoop.etl.io.DataWriter;
|
||||
import org.apache.sqoop.job.etl.Extractor;
|
||||
import org.apache.sqoop.job.etl.ExtractorContext;
|
||||
|
@ -40,8 +40,8 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.OutputFormat.SEQUENCE_FILE;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.OutputFormat.TEXT_FILE;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.ToFormat.SEQUENCE_FILE;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.ToFormat.TEXT_FILE;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestExtractor extends TestHdfsBase {
|
||||
|
@ -49,12 +49,12 @@ public class TestExtractor extends TestHdfsBase {
|
|||
private static final int NUMBER_OF_FILES = 5;
|
||||
private static final int NUMBER_OF_ROWS_PER_FILE = 1000;
|
||||
|
||||
private OutputFormat outputFileType;
|
||||
private ToFormat outputFileType;
|
||||
private Class<? extends CompressionCodec> compressionClass;
|
||||
private final String inputDirectory;
|
||||
private Extractor extractor;
|
||||
|
||||
public TestExtractor(OutputFormat outputFileType,
|
||||
public TestExtractor(ToFormat outputFileType,
|
||||
Class<? extends CompressionCodec> compressionClass)
|
||||
throws Exception {
|
||||
this.inputDirectory = INPUT_ROOT + getClass().getSimpleName();
|
||||
|
@ -130,7 +130,7 @@ public class TestExtractor extends TestHdfsBase {
|
|||
throw new AssertionError("Should not be writing object.");
|
||||
}
|
||||
}, null);
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
HdfsPartition partition = createPartition(FileUtils.listDir(inputDirectory));
|
||||
|
|
|
@ -27,9 +27,9 @@ import org.apache.hadoop.io.Text;
|
|||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.io.compress.CompressionCodecFactory;
|
||||
import org.apache.sqoop.common.PrefixContext;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.OutputCompression;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.OutputFormat;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToCompression;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToJobConfiguration;
|
||||
import org.apache.sqoop.etl.io.DataReader;
|
||||
import org.apache.sqoop.job.etl.Loader;
|
||||
|
@ -48,21 +48,21 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.OutputFormat.SEQUENCE_FILE;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.OutputFormat.TEXT_FILE;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.ToFormat.SEQUENCE_FILE;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.ToFormat.TEXT_FILE;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestLoader extends TestHdfsBase {
|
||||
private static final String INPUT_ROOT = System.getProperty("maven.build.directory", "/tmp") + "/sqoop/warehouse/";
|
||||
private static final int NUMBER_OF_ROWS_PER_FILE = 1000;
|
||||
|
||||
private OutputFormat outputFormat;
|
||||
private OutputCompression compression;
|
||||
private ToFormat outputFormat;
|
||||
private ToCompression compression;
|
||||
private final String outputDirectory;
|
||||
private Loader loader;
|
||||
|
||||
public TestLoader(OutputFormat outputFormat,
|
||||
OutputCompression compression)
|
||||
public TestLoader(ToFormat outputFormat,
|
||||
ToCompression compression)
|
||||
throws Exception {
|
||||
this.outputDirectory = INPUT_ROOT + getClass().getSimpleName();
|
||||
this.outputFormat = outputFormat;
|
||||
|
@ -73,10 +73,10 @@ public class TestLoader extends TestHdfsBase {
|
|||
@Parameterized.Parameters
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> parameters = new ArrayList<Object[]>();
|
||||
for (OutputCompression compression : new OutputCompression[]{
|
||||
OutputCompression.DEFAULT,
|
||||
OutputCompression.BZIP2,
|
||||
OutputCompression.NONE
|
||||
for (ToCompression compression : new ToCompression[]{
|
||||
ToCompression.DEFAULT,
|
||||
ToCompression.BZIP2,
|
||||
ToCompression.NONE
|
||||
}) {
|
||||
for (Object outputFileType : new Object[]{TEXT_FILE, SEQUENCE_FILE}) {
|
||||
parameters.add(new Object[]{outputFileType, compression});
|
||||
|
@ -121,11 +121,11 @@ public class TestLoader extends TestHdfsBase {
|
|||
return null;
|
||||
}
|
||||
}, null);
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
ToJobConfiguration jobConf = new ToJobConfiguration();
|
||||
jobConf.output.outputDirectory = outputDirectory;
|
||||
jobConf.output.compression = compression;
|
||||
jobConf.output.outputFormat = outputFormat;
|
||||
jobConf.toJobConfig.outputDirectory = outputDirectory;
|
||||
jobConf.toJobConfig.compression = compression;
|
||||
jobConf.toJobConfig.outputFormat = outputFormat;
|
||||
Path outputPath = new Path(outputDirectory);
|
||||
|
||||
loader.load(context, connConf, jobConf);
|
||||
|
|
|
@ -22,9 +22,9 @@ import org.apache.hadoop.io.compress.BZip2Codec;
|
|||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.io.compress.DefaultCodec;
|
||||
import org.apache.sqoop.common.PrefixContext;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.OutputFormat;
|
||||
import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
|
||||
import org.apache.sqoop.job.etl.Partition;
|
||||
import org.apache.sqoop.job.etl.Partitioner;
|
||||
import org.apache.sqoop.job.etl.PartitionerContext;
|
||||
|
@ -39,7 +39,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.OutputFormat.*;
|
||||
import static org.apache.sqoop.connector.hdfs.configuration.ToFormat.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
|
@ -48,13 +48,13 @@ public class TestPartitioner extends TestHdfsBase {
|
|||
private static final int NUMBER_OF_FILES = 5;
|
||||
private static final int NUMBER_OF_ROWS_PER_FILE = 1000;
|
||||
|
||||
private OutputFormat outputFileType;
|
||||
private ToFormat outputFileType;
|
||||
private Class<? extends CompressionCodec> compressionClass;
|
||||
private Partitioner partitioner;
|
||||
|
||||
private final String inputDirectory;
|
||||
|
||||
public TestPartitioner(OutputFormat outputFileType, Class<? extends CompressionCodec> compressionClass) {
|
||||
public TestPartitioner(ToFormat outputFileType, Class<? extends CompressionCodec> compressionClass) {
|
||||
this.inputDirectory = INPUT_ROOT + getClass().getSimpleName();
|
||||
this.outputFileType = outputFileType;
|
||||
this.compressionClass = compressionClass;
|
||||
|
@ -97,10 +97,10 @@ public class TestPartitioner extends TestHdfsBase {
|
|||
Configuration conf = new Configuration();
|
||||
PrefixContext prefixContext = new PrefixContext(conf, "org.apache.sqoop.job.connector.from.context.");
|
||||
PartitionerContext context = new PartitionerContext(prefixContext, 5, null);
|
||||
ConnectionConfiguration connConf = new ConnectionConfiguration();
|
||||
LinkConfiguration connConf = new LinkConfiguration();
|
||||
FromJobConfiguration jobConf = new FromJobConfiguration();
|
||||
|
||||
jobConf.input.inputDirectory = inputDirectory;
|
||||
jobConf.fromJobConfig.inputDirectory = inputDirectory;
|
||||
|
||||
List<Partition> partitions = partitioner.getPartitions(context, connConf, jobConf);
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.io.IOException;
|
|||
|
||||
/**
|
||||
* Abstract class representing a pluggable intermediate data format the Sqoop
|
||||
* framework will use to move data to/from the connector. All intermediate
|
||||
* driver will use to move data to/from the connector. All intermediate
|
||||
* data formats are expected to have an internal/native implementation,
|
||||
* but also should minimally be able to return a text (CSV) version of the
|
||||
* data. The data format should also be able to return the data as an object
|
||||
|
|
|
@ -106,7 +106,7 @@ public final class ConnectorHandler {
|
|||
}
|
||||
|
||||
MConnectionForms connectionForms = new MConnectionForms(
|
||||
FormUtils.toForms(connector.getConnectionConfigurationClass()));
|
||||
FormUtils.toForms(connector.getLinkConfigurationClass()));
|
||||
|
||||
String connectorVersion = connector.getVersion();
|
||||
|
||||
|
|
|
@ -73,8 +73,8 @@ public final class ConfigurationConstants {
|
|||
public static final String CONNECTOR_AUTO_UPGRADE =
|
||||
"org.apache.sqoop.connector.autoupgrade";
|
||||
|
||||
public static final String FRAMEWORK_AUTO_UPGRADE =
|
||||
"org.apache.sqoop.framework.autoupgrade";
|
||||
public static final String DRIVER_AUTO_UPGRADE =
|
||||
"org.apache.sqoop.driver.autoupgrade";
|
||||
|
||||
/**
|
||||
* Enable Sqoop App to kill Tomcat in case that it will fail to load.
|
||||
|
|
|
@ -20,8 +20,8 @@ package org.apache.sqoop.core;
|
|||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.audit.AuditLoggerManager;
|
||||
import org.apache.sqoop.connector.ConnectorManager;
|
||||
import org.apache.sqoop.framework.FrameworkManager;
|
||||
import org.apache.sqoop.framework.JobManager;
|
||||
import org.apache.sqoop.driver.Driver;
|
||||
import org.apache.sqoop.driver.JobManager;
|
||||
import org.apache.sqoop.repository.RepositoryManager;
|
||||
|
||||
/**
|
||||
|
@ -34,7 +34,7 @@ public class SqoopServer {
|
|||
public static void destroy() {
|
||||
LOG.info("Shutting down Sqoop server");
|
||||
JobManager.getInstance().destroy();
|
||||
FrameworkManager.getInstance().destroy();
|
||||
Driver.getInstance().destroy();
|
||||
ConnectorManager.getInstance().destroy();
|
||||
RepositoryManager.getInstance().destroy();
|
||||
AuditLoggerManager.getInstance().destroy();
|
||||
|
@ -49,7 +49,7 @@ public class SqoopServer {
|
|||
AuditLoggerManager.getInstance().initialize();
|
||||
RepositoryManager.getInstance().initialize();
|
||||
ConnectorManager.getInstance().initialize();
|
||||
FrameworkManager.getInstance().initialize();
|
||||
Driver.getInstance().initialize();
|
||||
JobManager.getInstance().initialize();
|
||||
LOG.info("Sqoop server has successfully boot up");
|
||||
} catch (Exception ex) {
|
||||
|
|
|
@ -15,30 +15,33 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||
import org.apache.sqoop.core.ConfigurationConstants;
|
||||
import org.apache.sqoop.core.Reconfigurable;
|
||||
import org.apache.sqoop.core.SqoopConfiguration;
|
||||
import org.apache.sqoop.core.SqoopConfiguration.CoreConfigurationListener;
|
||||
import org.apache.sqoop.framework.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.framework.configuration.JobConfiguration;
|
||||
import org.apache.sqoop.model.*;
|
||||
import org.apache.sqoop.repository.RepositoryManager;
|
||||
import org.apache.sqoop.validation.Validator;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import java.util.Locale;
|
||||
import java.util.ResourceBundle;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.connector.spi.RepositoryUpgrader;
|
||||
import org.apache.sqoop.core.ConfigurationConstants;
|
||||
import org.apache.sqoop.core.Reconfigurable;
|
||||
import org.apache.sqoop.core.SqoopConfiguration;
|
||||
import org.apache.sqoop.core.SqoopConfiguration.CoreConfigurationListener;
|
||||
import org.apache.sqoop.driver.configuration.JobConfiguration;
|
||||
import org.apache.sqoop.driver.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.model.FormUtils;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MDriverConfig;
|
||||
import org.apache.sqoop.model.MJobForms;
|
||||
import org.apache.sqoop.repository.RepositoryManager;
|
||||
import org.apache.sqoop.validation.Validator;
|
||||
|
||||
/**
|
||||
* Manager for Sqoop framework itself.
|
||||
* Sqoop driver that manages the job lifecyle
|
||||
*
|
||||
* All Sqoop internals are handled in this class:
|
||||
* * Submission engine
|
||||
* * Execution engine
|
||||
* * Framework metadata
|
||||
* * Driver config
|
||||
*
|
||||
* Current implementation of entire submission engine is using repository
|
||||
* for keeping track of running submissions. Thus, server might be restarted at
|
||||
|
@ -46,17 +49,17 @@ import java.util.ResourceBundle;
|
|||
* be the fastest way and we might want to introduce internal structures for
|
||||
* running jobs in case that this approach will be too slow.
|
||||
*/
|
||||
public class FrameworkManager implements Reconfigurable {
|
||||
public class Driver implements Reconfigurable {
|
||||
|
||||
/**
|
||||
* Logger object.
|
||||
*/
|
||||
private static final Logger LOG = Logger.getLogger(FrameworkManager.class);
|
||||
private static final Logger LOG = Logger.getLogger(Driver.class);
|
||||
|
||||
/**
|
||||
* Private instance to singleton of this class.
|
||||
*/
|
||||
private static FrameworkManager instance;
|
||||
private static Driver instance;
|
||||
|
||||
/**
|
||||
* Create default object by default.
|
||||
|
@ -64,7 +67,7 @@ public class FrameworkManager implements Reconfigurable {
|
|||
* Every Sqoop server application needs one so this should not be performance issue.
|
||||
*/
|
||||
static {
|
||||
instance = new FrameworkManager();
|
||||
instance = new Driver();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -72,7 +75,7 @@ public class FrameworkManager implements Reconfigurable {
|
|||
*
|
||||
* @return Current instance
|
||||
*/
|
||||
public static FrameworkManager getInstance() {
|
||||
public static Driver getInstance() {
|
||||
return instance;
|
||||
}
|
||||
|
||||
|
@ -84,14 +87,14 @@ public class FrameworkManager implements Reconfigurable {
|
|||
*
|
||||
* @param newInstance New instance
|
||||
*/
|
||||
public static void setInstance(FrameworkManager newInstance) {
|
||||
public static void setInstance(Driver newInstance) {
|
||||
instance = newInstance;
|
||||
}
|
||||
|
||||
/**
|
||||
* Framework metadata structures in MForm format
|
||||
* Driver config structure
|
||||
*/
|
||||
private MFramework mFramework;
|
||||
private MDriverConfig mDriverConfig;
|
||||
|
||||
/**
|
||||
* Validator instance
|
||||
|
@ -99,80 +102,78 @@ public class FrameworkManager implements Reconfigurable {
|
|||
private final Validator validator;
|
||||
|
||||
/**
|
||||
* Upgrader instance
|
||||
* Driver config upgrader instance
|
||||
*/
|
||||
private final MetadataUpgrader upgrader;
|
||||
private final RepositoryUpgrader driverConfigUpgrader;
|
||||
|
||||
/**
|
||||
* Default framework auto upgrade option value
|
||||
* Default driver config auto upgrade option value
|
||||
*/
|
||||
private static final boolean DEFAULT_AUTO_UPGRADE = false;
|
||||
|
||||
public static final String CURRENT_FRAMEWORK_VERSION = "1";
|
||||
public static final String CURRENT_DRIVER_VERSION = "1";
|
||||
|
||||
public Class getJobConfigurationClass() {
|
||||
return JobConfiguration.class;
|
||||
}
|
||||
|
||||
public Class getConnectionConfigurationClass() {
|
||||
return ConnectionConfiguration.class;
|
||||
public Class getLinkConfigurationClass() {
|
||||
return LinkConfiguration.class;
|
||||
}
|
||||
|
||||
public FrameworkManager() {
|
||||
public Driver() {
|
||||
MConnectionForms connectionForms = new MConnectionForms(
|
||||
FormUtils.toForms(getConnectionConfigurationClass())
|
||||
FormUtils.toForms(getLinkConfigurationClass())
|
||||
);
|
||||
mFramework = new MFramework(connectionForms, new MJobForms(FormUtils.toForms(getJobConfigurationClass())),
|
||||
CURRENT_FRAMEWORK_VERSION);
|
||||
mDriverConfig = new MDriverConfig(connectionForms, new MJobForms(FormUtils.toForms(getJobConfigurationClass())),
|
||||
CURRENT_DRIVER_VERSION);
|
||||
|
||||
// Build validator
|
||||
validator = new FrameworkValidator();
|
||||
|
||||
validator = new DriverValidator();
|
||||
// Build upgrader
|
||||
upgrader = new FrameworkMetadataUpgrader();
|
||||
driverConfigUpgrader = new DriverConfigUpgrader();
|
||||
}
|
||||
|
||||
public synchronized void initialize() {
|
||||
initialize(SqoopConfiguration.getInstance().getContext().getBoolean(ConfigurationConstants.FRAMEWORK_AUTO_UPGRADE, DEFAULT_AUTO_UPGRADE));
|
||||
initialize(SqoopConfiguration.getInstance().getContext().getBoolean(ConfigurationConstants.DRIVER_AUTO_UPGRADE, DEFAULT_AUTO_UPGRADE));
|
||||
}
|
||||
|
||||
public synchronized void initialize(boolean autoUpgrade) {
|
||||
LOG.trace("Begin submission engine manager initialization");
|
||||
LOG.trace("Begin Driver Config initialization");
|
||||
|
||||
// Register framework metadata in repository
|
||||
mFramework = RepositoryManager.getInstance().getRepository().registerFramework(mFramework, autoUpgrade);
|
||||
// Register driver config in repository
|
||||
mDriverConfig = RepositoryManager.getInstance().getRepository().registerDriverConfig(mDriverConfig, autoUpgrade);
|
||||
|
||||
SqoopConfiguration.getInstance().getProvider().registerListener(new CoreConfigurationListener(this));
|
||||
|
||||
LOG.info("Submission manager initialized: OK");
|
||||
LOG.info("Driver Config initialized: OK");
|
||||
}
|
||||
|
||||
public synchronized void destroy() {
|
||||
LOG.trace("Begin submission engine manager destroy");
|
||||
LOG.trace("Begin Driver Config destroy");
|
||||
}
|
||||
|
||||
public Validator getValidator() {
|
||||
return validator;
|
||||
}
|
||||
|
||||
public MetadataUpgrader getMetadataUpgrader() {
|
||||
return upgrader;
|
||||
public RepositoryUpgrader getDriverConfigRepositoryUpgrader() {
|
||||
return driverConfigUpgrader;
|
||||
}
|
||||
|
||||
public MFramework getFramework() {
|
||||
return mFramework;
|
||||
public MDriverConfig getDriverConfig() {
|
||||
return mDriverConfig;
|
||||
}
|
||||
|
||||
public ResourceBundle getBundle(Locale locale) {
|
||||
return ResourceBundle.getBundle(
|
||||
FrameworkConstants.RESOURCE_BUNDLE_NAME, locale);
|
||||
return ResourceBundle.getBundle(DriverConstants.DRIVER_CONFIG_BUNDLE, locale);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configurationChanged() {
|
||||
LOG.info("Begin framework manager reconfiguring");
|
||||
// If there are configuration options for FrameworkManager,
|
||||
LOG.info("Begin Driver reconfiguring");
|
||||
// If there are configuration options for Driver,
|
||||
// implement the reconfiguration procedure right here.
|
||||
LOG.info("Framework manager reconfigured");
|
||||
LOG.info("Driver reconfigured");
|
||||
}
|
||||
}
|
|
@ -16,11 +16,11 @@
|
|||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||
import org.apache.sqoop.connector.spi.RepositoryUpgrader;
|
||||
import org.apache.sqoop.model.MConnectionForms;
|
||||
import org.apache.sqoop.model.MForm;
|
||||
import org.apache.sqoop.model.MInput;
|
||||
|
@ -30,9 +30,9 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class FrameworkMetadataUpgrader extends MetadataUpgrader{
|
||||
public class DriverConfigUpgrader extends RepositoryUpgrader{
|
||||
|
||||
private static final Logger LOG = Logger.getLogger(FrameworkMetadataUpgrader.class);
|
||||
private static final Logger LOG = Logger.getLogger(DriverConfigUpgrader.class);
|
||||
|
||||
@Override
|
||||
public void upgrade(MConnectionForms original,
|
||||
|
@ -61,7 +61,7 @@ public class FrameworkMetadataUpgrader extends MetadataUpgrader{
|
|||
MForm originalForm = formMap.get(form.getName());
|
||||
if(originalForm == null) {
|
||||
LOG.warn("Form: " + form.getName() + " not present in old " +
|
||||
"framework metadata. So it will not be transferred by the upgrader.");
|
||||
"driver config. So it will not be transferred by the upgrader.");
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -71,7 +71,7 @@ public class FrameworkMetadataUpgrader extends MetadataUpgrader{
|
|||
input.setValue(originalInput.getValue());
|
||||
} catch (SqoopException ex) {
|
||||
LOG.warn("Input: " + input.getName() + " not present in old " +
|
||||
"framework metadata. So it will not be transferred by the upgrader.");
|
||||
"driver config. So it will not be transferred by the upgrader.");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -15,16 +15,16 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.sqoop.core.ConfigurationConstants;
|
||||
|
||||
/**
|
||||
* Constants that are used in framework module.
|
||||
* Constants that are used in driver module.
|
||||
*/
|
||||
public final class FrameworkConstants {
|
||||
public final class DriverConstants {
|
||||
|
||||
// Sqoop configuration constants
|
||||
// Sqoop driver constants
|
||||
|
||||
public static final String PREFIX_SUBMISSION_CONFIG =
|
||||
ConfigurationConstants.PREFIX_GLOBAL_CONFIG + "submission.";
|
||||
|
@ -59,11 +59,10 @@ public final class FrameworkConstants {
|
|||
public static final String PREFIX_EXECUTION_ENGINE_CONFIG =
|
||||
SYSCFG_EXECUTION_ENGINE + ".";
|
||||
|
||||
// Bundle names
|
||||
// Bundle name
|
||||
public static final String DRIVER_CONFIG_BUNDLE = "driver-config";
|
||||
|
||||
public static final String RESOURCE_BUNDLE_NAME = "framework-resources";
|
||||
|
||||
private FrameworkConstants() {
|
||||
private DriverConstants() {
|
||||
// Instantiation of this class is prohibited
|
||||
}
|
||||
}
|
|
@ -15,44 +15,44 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.sqoop.common.ErrorCode;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public enum FrameworkError implements ErrorCode {
|
||||
public enum DriverError implements ErrorCode {
|
||||
|
||||
FRAMEWORK_0000("Metadata are not registered in repository"),
|
||||
DRIVER_0000("Metadata are not registered in repository"),
|
||||
|
||||
FRAMEWORK_0001("Invalid submission engine"),
|
||||
DRIVER_0001("Invalid submission engine"),
|
||||
|
||||
FRAMEWORK_0002("Given job is already running"),
|
||||
DRIVER_0002("Given job is already running"),
|
||||
|
||||
FRAMEWORK_0003("Given job is not running"),
|
||||
DRIVER_0003("Given job is not running"),
|
||||
|
||||
FRAMEWORK_0004("Unknown job id"),
|
||||
DRIVER_0004("Unknown job id"),
|
||||
|
||||
FRAMEWORK_0005("Unsupported job type"),
|
||||
DRIVER_0005("Unsupported job type"),
|
||||
|
||||
FRAMEWORK_0006("Can't bootstrap job"),
|
||||
DRIVER_0006("Can't bootstrap job"),
|
||||
|
||||
FRAMEWORK_0007("Invalid execution engine"),
|
||||
DRIVER_0007("Invalid execution engine"),
|
||||
|
||||
FRAMEWORK_0008("Invalid combination of submission and execution engines"),
|
||||
DRIVER_0008("Invalid combination of submission and execution engines"),
|
||||
|
||||
FRAMEWORK_0009("Job has been disabled. Cannot submit this job."),
|
||||
DRIVER_0009("Job has been disabled. Cannot submit this job."),
|
||||
|
||||
FRAMEWORK_0010("Connection for this job has been disabled. Cannot submit this job."),
|
||||
DRIVER_0010("Connection for this job has been disabled. Cannot submit this job."),
|
||||
|
||||
FRAMEWORK_0011("Connector does not support direction. Cannot submit this job."),
|
||||
DRIVER_0011("Connector does not support direction. Cannot submit this job."),
|
||||
|
||||
;
|
||||
|
||||
private final String message;
|
||||
|
||||
private FrameworkError(String message) {
|
||||
private DriverError(String message) {
|
||||
this.message = message;
|
||||
}
|
||||
|
|
@ -15,20 +15,20 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.sqoop.framework.configuration.ConnectionConfiguration;
|
||||
import org.apache.sqoop.framework.configuration.JobConfiguration;
|
||||
import org.apache.sqoop.framework.configuration.ThrottlingForm;
|
||||
import org.apache.sqoop.driver.configuration.LinkConfiguration;
|
||||
import org.apache.sqoop.driver.configuration.JobConfiguration;
|
||||
import org.apache.sqoop.driver.configuration.ThrottlingForm;
|
||||
import org.apache.sqoop.validation.Status;
|
||||
import org.apache.sqoop.validation.Validation;
|
||||
import org.apache.sqoop.validation.Validator;
|
||||
|
||||
public class FrameworkValidator extends Validator {
|
||||
public class DriverValidator extends Validator {
|
||||
@Override
|
||||
public Validation validateConnection(Object connectionConfiguration) {
|
||||
Validation validation = new Validation(ConnectionConfiguration.class);
|
||||
// No validation on connection object
|
||||
public Validation validateLink(Object linkConfiguration) {
|
||||
Validation validation = new Validation(LinkConfiguration.class);
|
||||
// No validation on link object
|
||||
return validation;
|
||||
}
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.sqoop.common.ImmutableContext;
|
||||
|
|
@ -1,5 +1,3 @@
|
|||
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
|
@ -17,36 +15,40 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.common.MapContext;
|
||||
import org.apache.sqoop.common.SqoopException;
|
||||
import org.apache.sqoop.connector.ConnectorManager;
|
||||
import org.apache.sqoop.framework.configuration.JobConfiguration;
|
||||
import org.apache.sqoop.request.HttpEventContext;
|
||||
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
||||
import org.apache.sqoop.connector.spi.SqoopConnector;
|
||||
import org.apache.sqoop.core.Reconfigurable;
|
||||
import org.apache.sqoop.core.SqoopConfiguration;
|
||||
import org.apache.sqoop.core.SqoopConfiguration.CoreConfigurationListener;
|
||||
import org.apache.sqoop.job.etl.*;
|
||||
import org.apache.sqoop.driver.configuration.JobConfiguration;
|
||||
import org.apache.sqoop.job.etl.Destroyer;
|
||||
import org.apache.sqoop.job.etl.DestroyerContext;
|
||||
import org.apache.sqoop.job.etl.Initializer;
|
||||
import org.apache.sqoop.job.etl.InitializerContext;
|
||||
import org.apache.sqoop.job.etl.Transferable;
|
||||
import org.apache.sqoop.model.FormUtils;
|
||||
import org.apache.sqoop.model.MConnection;
|
||||
import org.apache.sqoop.model.MJob;
|
||||
import org.apache.sqoop.model.MLink;
|
||||
import org.apache.sqoop.model.MSubmission;
|
||||
import org.apache.sqoop.repository.Repository;
|
||||
import org.apache.sqoop.repository.RepositoryManager;
|
||||
import org.apache.sqoop.request.HttpEventContext;
|
||||
import org.apache.sqoop.schema.Schema;
|
||||
import org.apache.sqoop.submission.SubmissionStatus;
|
||||
import org.apache.sqoop.submission.counter.Counters;
|
||||
import org.apache.sqoop.utils.ClassUtils;
|
||||
import org.json.simple.JSONValue;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
public class JobManager implements Reconfigurable {
|
||||
/**
|
||||
* Logger object.
|
||||
|
@ -147,7 +149,7 @@ public class JobManager implements Reconfigurable {
|
|||
/**
|
||||
* Base notification URL.
|
||||
*
|
||||
* Framework manager will always add job id.
|
||||
* Driver manager will always add job id.
|
||||
*/
|
||||
private String notificationBaseUrl;
|
||||
|
||||
|
@ -207,26 +209,26 @@ public class JobManager implements Reconfigurable {
|
|||
|
||||
// Let's load configured submission engine
|
||||
String submissionEngineClassName =
|
||||
context.getString(FrameworkConstants.SYSCFG_SUBMISSION_ENGINE);
|
||||
context.getString(DriverConstants.SYSCFG_SUBMISSION_ENGINE);
|
||||
|
||||
submissionEngine = (SubmissionEngine) ClassUtils
|
||||
.instantiate(submissionEngineClassName);
|
||||
if (submissionEngine == null) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0001,
|
||||
throw new SqoopException(DriverError.DRIVER_0001,
|
||||
submissionEngineClassName);
|
||||
}
|
||||
|
||||
submissionEngine.initialize(context,
|
||||
FrameworkConstants.PREFIX_SUBMISSION_ENGINE_CONFIG);
|
||||
DriverConstants.PREFIX_SUBMISSION_ENGINE_CONFIG);
|
||||
|
||||
// Execution engine
|
||||
String executionEngineClassName =
|
||||
context.getString(FrameworkConstants.SYSCFG_EXECUTION_ENGINE);
|
||||
context.getString(DriverConstants.SYSCFG_EXECUTION_ENGINE);
|
||||
|
||||
executionEngine = (ExecutionEngine) ClassUtils
|
||||
.instantiate(executionEngineClassName);
|
||||
if (executionEngine == null) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0007,
|
||||
throw new SqoopException(DriverError.DRIVER_0007,
|
||||
executionEngineClassName);
|
||||
}
|
||||
|
||||
|
@ -234,19 +236,19 @@ public class JobManager implements Reconfigurable {
|
|||
// submission engine and execution engine
|
||||
if (!submissionEngine
|
||||
.isExecutionEngineSupported(executionEngine.getClass())) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0008);
|
||||
throw new SqoopException(DriverError.DRIVER_0008);
|
||||
}
|
||||
|
||||
executionEngine.initialize(context,
|
||||
FrameworkConstants.PREFIX_EXECUTION_ENGINE_CONFIG);
|
||||
DriverConstants.PREFIX_EXECUTION_ENGINE_CONFIG);
|
||||
|
||||
// Set up worker threads
|
||||
purgeThreshold = context.getLong(
|
||||
FrameworkConstants.SYSCFG_SUBMISSION_PURGE_THRESHOLD,
|
||||
DriverConstants.SYSCFG_SUBMISSION_PURGE_THRESHOLD,
|
||||
DEFAULT_PURGE_THRESHOLD
|
||||
);
|
||||
purgeSleep = context.getLong(
|
||||
FrameworkConstants.SYSCFG_SUBMISSION_PURGE_SLEEP,
|
||||
DriverConstants.SYSCFG_SUBMISSION_PURGE_SLEEP,
|
||||
DEFAULT_PURGE_SLEEP
|
||||
);
|
||||
|
||||
|
@ -254,7 +256,7 @@ public class JobManager implements Reconfigurable {
|
|||
purgeThread.start();
|
||||
|
||||
updateSleep = context.getLong(
|
||||
FrameworkConstants.SYSCFG_SUBMISSION_UPDATE_SLEEP,
|
||||
DriverConstants.SYSCFG_SUBMISSION_UPDATE_SLEEP,
|
||||
DEFAULT_UPDATE_SLEEP
|
||||
);
|
||||
|
||||
|
@ -279,7 +281,7 @@ public class JobManager implements Reconfigurable {
|
|||
MSubmission lastSubmission = RepositoryManager.getInstance().getRepository()
|
||||
.findSubmissionLastForJob(jobId);
|
||||
if (lastSubmission != null && lastSubmission.getStatus().isRunning()) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0002, "Job with id " + jobId);
|
||||
throw new SqoopException(DriverError.DRIVER_0002, "Job with id " + jobId);
|
||||
}
|
||||
// TODO(Abe): Call multiple destroyers.
|
||||
// TODO(jarcec): We might need to catch all exceptions here to ensure
|
||||
|
@ -300,8 +302,8 @@ public class JobManager implements Reconfigurable {
|
|||
MJob job = getJob(jobId);
|
||||
|
||||
// get from/to connections for the job
|
||||
MConnection fromConnection = getConnection(job.getConnectionId(Direction.FROM));
|
||||
MConnection toConnection = getConnection(job.getConnectionId(Direction.TO));
|
||||
MLink fromConnection = getLink(job.getLinkId(Direction.FROM));
|
||||
MLink toConnection = getLink(job.getLinkId(Direction.TO));
|
||||
|
||||
// get from/to connectors for the connection
|
||||
SqoopConnector fromConnector = getConnector(fromConnection.getConnectorId());
|
||||
|
@ -311,12 +313,12 @@ public class JobManager implements Reconfigurable {
|
|||
|
||||
// Transform config to fromConnector specific classes
|
||||
Object fromConnectionConfig = ClassUtils.instantiate(fromConnector
|
||||
.getConnectionConfigurationClass());
|
||||
.getLinkConfigurationClass());
|
||||
FormUtils.fromForms(fromConnection.getConnectorPart().getForms(), fromConnectionConfig);
|
||||
|
||||
// Transform config to toConnector specific classes
|
||||
Object toConnectorConfig = ClassUtils
|
||||
.instantiate(toConnector.getConnectionConfigurationClass());
|
||||
.instantiate(toConnector.getLinkConfigurationClass());
|
||||
FormUtils.fromForms(toConnection.getConnectorPart().getForms(), toConnectorConfig);
|
||||
|
||||
Object fromJob = ClassUtils.instantiate(fromConnector.getJobConfigurationClass(Direction.FROM));
|
||||
|
@ -327,15 +329,15 @@ public class JobManager implements Reconfigurable {
|
|||
|
||||
// Transform framework specific configs
|
||||
// Q(VB) : Aren't the following 2 exactly the same?
|
||||
Object fromFrameworkConnection = ClassUtils.instantiate(FrameworkManager.getInstance()
|
||||
.getConnectionConfigurationClass());
|
||||
FormUtils.fromForms(fromConnection.getFrameworkPart().getForms(), fromFrameworkConnection);
|
||||
Object fromDriverConnection = ClassUtils.instantiate(Driver.getInstance()
|
||||
.getLinkConfigurationClass());
|
||||
FormUtils.fromForms(fromConnection.getFrameworkPart().getForms(), fromDriverConnection);
|
||||
|
||||
Object toFrameworkConnection = ClassUtils.instantiate(FrameworkManager.getInstance()
|
||||
.getConnectionConfigurationClass());
|
||||
FormUtils.fromForms(toConnection.getFrameworkPart().getForms(), toFrameworkConnection);
|
||||
Object toDriverConnection = ClassUtils.instantiate(Driver.getInstance()
|
||||
.getLinkConfigurationClass());
|
||||
FormUtils.fromForms(toConnection.getFrameworkPart().getForms(), toDriverConnection);
|
||||
|
||||
Object frameworkJob = ClassUtils.instantiate(FrameworkManager.getInstance()
|
||||
Object frameworkJob = ClassUtils.instantiate(Driver.getInstance()
|
||||
.getJobConfigurationClass());
|
||||
FormUtils.fromForms(job.getFrameworkPart().getForms(), frameworkJob);
|
||||
|
||||
|
@ -345,14 +347,14 @@ public class JobManager implements Reconfigurable {
|
|||
jobRequest.setSummary(submission);
|
||||
jobRequest.setConnector(Direction.FROM, fromConnector);
|
||||
jobRequest.setConnector(Direction.TO, toConnector);
|
||||
jobRequest.setConnectorConnectionConfig(Direction.FROM, fromConnectionConfig);
|
||||
jobRequest.setConnectorConnectionConfig(Direction.TO, toConnectorConfig);
|
||||
jobRequest.setConnectorLinkConfig(Direction.FROM, fromConnectionConfig);
|
||||
jobRequest.setConnectorLinkConfig(Direction.TO, toConnectorConfig);
|
||||
jobRequest.setConnectorJobConfig(Direction.FROM, fromJob);
|
||||
jobRequest.setConnectorJobConfig(Direction.TO, toJob);
|
||||
// TODO(Abe): Should we actually have 2 different Framework Connection config objects?
|
||||
jobRequest.setFrameworkConnectionConfig(Direction.FROM, fromFrameworkConnection);
|
||||
jobRequest.setFrameworkConnectionConfig(Direction.TO, toFrameworkConnection);
|
||||
jobRequest.setConfigFrameworkJob(frameworkJob);
|
||||
// TODO(Abe): Should we actually have 2 different Driver Connection config objects?
|
||||
jobRequest.setFrameworkLinkConfig(Direction.FROM, fromDriverConnection);
|
||||
jobRequest.setFrameworkLinkConfig(Direction.TO, toDriverConnection);
|
||||
jobRequest.setFrameworkJobConfig(frameworkJob);
|
||||
jobRequest.setJobName(job.getName());
|
||||
jobRequest.setJobId(job.getPersistenceId());
|
||||
jobRequest.setNotificationUrl(notificationBaseUrl + jobId);
|
||||
|
@ -395,7 +397,7 @@ public class JobManager implements Reconfigurable {
|
|||
// sqoop-common
|
||||
jobRequest.addJarForClass(MapContext.class);
|
||||
// sqoop-core
|
||||
jobRequest.addJarForClass(FrameworkManager.class);
|
||||
jobRequest.addJarForClass(Driver.class);
|
||||
// sqoop-spi
|
||||
jobRequest.addJarForClass(SqoopConnector.class);
|
||||
// Execution engine jar
|
||||
|
@ -418,29 +420,29 @@ public class JobManager implements Reconfigurable {
|
|||
void validateSupportedDirection(SqoopConnector connector, Direction direction) {
|
||||
// Make sure that connector supports the given direction
|
||||
if (!connector.getSupportedDirections().contains(direction)) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0011, "Connector: "
|
||||
throw new SqoopException(DriverError.DRIVER_0011, "Connector: "
|
||||
+ connector.getClass().getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
MConnection getConnection(long connectionId) {
|
||||
MConnection connection = RepositoryManager.getInstance().getRepository()
|
||||
.findConnection(connectionId);
|
||||
if (!connection.getEnabled()) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0010, "Connection id: "
|
||||
+ connection.getPersistenceId());
|
||||
MLink getLink(long linkId) {
|
||||
MLink link = RepositoryManager.getInstance().getRepository()
|
||||
.findLink(linkId);
|
||||
if (!link.getEnabled()) {
|
||||
throw new SqoopException(DriverError.DRIVER_0010, "Connection id: "
|
||||
+ link.getPersistenceId());
|
||||
}
|
||||
return connection;
|
||||
return link;
|
||||
}
|
||||
|
||||
MJob getJob(long jobId) {
|
||||
MJob job = RepositoryManager.getInstance().getRepository().findJob(jobId);
|
||||
if (job == null) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0004, "Unknown job id: " + jobId);
|
||||
throw new SqoopException(DriverError.DRIVER_0004, "Unknown job id: " + jobId);
|
||||
}
|
||||
|
||||
if (!job.getEnabled()) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0009, "Job id: " + job.getPersistenceId());
|
||||
throw new SqoopException(DriverError.DRIVER_0009, "Job id: " + job.getPersistenceId());
|
||||
}
|
||||
return job;
|
||||
}
|
||||
|
@ -450,25 +452,25 @@ public class JobManager implements Reconfigurable {
|
|||
Initializer initializer = getConnectorInitializer(jobRequest, direction);
|
||||
|
||||
// Initializer context
|
||||
InitializerContext initializerContext = getInitializerContext(jobRequest, direction);
|
||||
InitializerContext initializerContext = getConnectorInitializerContext(jobRequest, direction);
|
||||
|
||||
// Initialize submission from the connector perspective
|
||||
initializer.initialize(initializerContext, jobRequest.getConnectorConnectionConfig(direction),
|
||||
initializer.initialize(initializerContext, jobRequest.getConnectorLinkConfig(direction),
|
||||
jobRequest.getConnectorJobConfig(direction));
|
||||
|
||||
// TODO(Abe): Alter behavior of Schema here.
|
||||
return initializer.getSchema(initializerContext,
|
||||
jobRequest.getConnectorConnectionConfig(direction),
|
||||
jobRequest.getConnectorLinkConfig(direction),
|
||||
jobRequest.getConnectorJobConfig(direction));
|
||||
}
|
||||
|
||||
private void addConnectorInitializerJars(JobRequest jobRequest, Direction direction) {
|
||||
|
||||
Initializer initializer = getConnectorInitializer(jobRequest, direction);
|
||||
InitializerContext initializerContext = getInitializerContext(jobRequest, direction);
|
||||
InitializerContext initializerContext = getConnectorInitializerContext(jobRequest, direction);
|
||||
// Add job specific jars to
|
||||
jobRequest.addJars(initializer.getJars(initializerContext,
|
||||
jobRequest.getConnectorConnectionConfig(direction),
|
||||
jobRequest.getConnectorLinkConfig(direction),
|
||||
jobRequest.getConnectorJobConfig(direction)));
|
||||
}
|
||||
|
||||
|
@ -478,18 +480,18 @@ public class JobManager implements Reconfigurable {
|
|||
Initializer initializer = (Initializer) ClassUtils.instantiate(initializerClass);
|
||||
|
||||
if (initializer == null) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
|
||||
throw new SqoopException(DriverError.DRIVER_0006,
|
||||
"Can't create connector initializer instance: " + initializerClass.getName());
|
||||
}
|
||||
return initializer;
|
||||
}
|
||||
|
||||
private InitializerContext getInitializerContext(JobRequest jobRequest, Direction direction) {
|
||||
private InitializerContext getConnectorInitializerContext(JobRequest jobRequest, Direction direction) {
|
||||
return new InitializerContext(jobRequest.getConnectorContext(direction));
|
||||
}
|
||||
|
||||
void prepareJob(JobRequest request) {
|
||||
JobConfiguration jobConfiguration = (JobConfiguration) request.getConfigFrameworkJob();
|
||||
JobConfiguration jobConfiguration = (JobConfiguration) request.getFrameworkJobConfig();
|
||||
// We're directly moving configured number of extractors and loaders to
|
||||
// underlying request object. In the future we might need to throttle this
|
||||
// count based on other running jobs to meet our SLAs.
|
||||
|
@ -514,12 +516,12 @@ public class JobManager implements Reconfigurable {
|
|||
Destroyer toDestroyer = (Destroyer) ClassUtils.instantiate(toDestroyerClass);
|
||||
|
||||
if (fromDestroyer == null) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
|
||||
throw new SqoopException(DriverError.DRIVER_0006,
|
||||
"Can't create toDestroyer instance: " + fromDestroyerClass.getName());
|
||||
}
|
||||
|
||||
if (toDestroyer == null) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
|
||||
throw new SqoopException(DriverError.DRIVER_0006,
|
||||
"Can't create toDestroyer instance: " + toDestroyerClass.getName());
|
||||
}
|
||||
|
||||
|
@ -532,9 +534,9 @@ public class JobManager implements Reconfigurable {
|
|||
.getToSchema());
|
||||
|
||||
// destroy submission from connector perspective
|
||||
fromDestroyer.destroy(fromDestroyerContext, request.getConnectorConnectionConfig(Direction.FROM),
|
||||
fromDestroyer.destroy(fromDestroyerContext, request.getConnectorLinkConfig(Direction.FROM),
|
||||
request.getConnectorJobConfig(Direction.FROM));
|
||||
toDestroyer.destroy(toDestroyerContext, request.getConnectorConnectionConfig(Direction.TO),
|
||||
toDestroyer.destroy(toDestroyerContext, request.getConnectorLinkConfig(Direction.TO),
|
||||
request.getConnectorJobConfig(Direction.TO));
|
||||
}
|
||||
|
||||
|
@ -544,7 +546,7 @@ public class JobManager implements Reconfigurable {
|
|||
MSubmission mSubmission = repository.findSubmissionLastForJob(jobId);
|
||||
|
||||
if (mSubmission == null || !mSubmission.getStatus().isRunning()) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0003, "Job with id " + jobId
|
||||
throw new SqoopException(DriverError.DRIVER_0003, "Job with id " + jobId
|
||||
+ " is not running");
|
||||
}
|
||||
submissionEngine.stop(mSubmission.getExternalId());
|
||||
|
@ -604,30 +606,30 @@ public class JobManager implements Reconfigurable {
|
|||
MapContext oldContext = SqoopConfiguration.getInstance().getOldContext();
|
||||
|
||||
String newSubmissionEngineClassName = newContext
|
||||
.getString(FrameworkConstants.SYSCFG_SUBMISSION_ENGINE);
|
||||
.getString(DriverConstants.SYSCFG_SUBMISSION_ENGINE);
|
||||
if (newSubmissionEngineClassName == null
|
||||
|| newSubmissionEngineClassName.trim().length() == 0) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0001,
|
||||
throw new SqoopException(DriverError.DRIVER_0001,
|
||||
newSubmissionEngineClassName);
|
||||
}
|
||||
|
||||
String oldSubmissionEngineClassName = oldContext
|
||||
.getString(FrameworkConstants.SYSCFG_SUBMISSION_ENGINE);
|
||||
.getString(DriverConstants.SYSCFG_SUBMISSION_ENGINE);
|
||||
if (!newSubmissionEngineClassName.equals(oldSubmissionEngineClassName)) {
|
||||
LOG.warn("Submission engine cannot be replaced at the runtime. " +
|
||||
"You might need to restart the server.");
|
||||
}
|
||||
|
||||
String newExecutionEngineClassName = newContext
|
||||
.getString(FrameworkConstants.SYSCFG_EXECUTION_ENGINE);
|
||||
.getString(DriverConstants.SYSCFG_EXECUTION_ENGINE);
|
||||
if (newExecutionEngineClassName == null
|
||||
|| newExecutionEngineClassName.trim().length() == 0) {
|
||||
throw new SqoopException(FrameworkError.FRAMEWORK_0007,
|
||||
throw new SqoopException(DriverError.DRIVER_0007,
|
||||
newExecutionEngineClassName);
|
||||
}
|
||||
|
||||
String oldExecutionEngineClassName = oldContext
|
||||
.getString(FrameworkConstants.SYSCFG_EXECUTION_ENGINE);
|
||||
.getString(DriverConstants.SYSCFG_EXECUTION_ENGINE);
|
||||
if (!newExecutionEngineClassName.equals(oldExecutionEngineClassName)) {
|
||||
LOG.warn("Execution engine cannot be replaced at the runtime. " +
|
||||
"You might need to restart the server.");
|
||||
|
@ -635,17 +637,17 @@ public class JobManager implements Reconfigurable {
|
|||
|
||||
// Set up worker threads
|
||||
purgeThreshold = newContext.getLong(
|
||||
FrameworkConstants.SYSCFG_SUBMISSION_PURGE_THRESHOLD,
|
||||
DriverConstants.SYSCFG_SUBMISSION_PURGE_THRESHOLD,
|
||||
DEFAULT_PURGE_THRESHOLD
|
||||
);
|
||||
purgeSleep = newContext.getLong(
|
||||
FrameworkConstants.SYSCFG_SUBMISSION_PURGE_SLEEP,
|
||||
DriverConstants.SYSCFG_SUBMISSION_PURGE_SLEEP,
|
||||
DEFAULT_PURGE_SLEEP
|
||||
);
|
||||
purgeThread.interrupt();
|
||||
|
||||
updateSleep = newContext.getLong(
|
||||
FrameworkConstants.SYSCFG_SUBMISSION_UPDATE_SLEEP,
|
||||
DriverConstants.SYSCFG_SUBMISSION_UPDATE_SLEEP,
|
||||
DEFAULT_UPDATE_SLEEP
|
||||
);
|
||||
updateThread.interrupt();
|
|
@ -15,7 +15,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.sqoop.common.Direction;
|
||||
import org.apache.sqoop.common.DirectionError;
|
||||
|
@ -76,13 +76,13 @@ public class JobRequest {
|
|||
/**
|
||||
* All configuration objects
|
||||
*/
|
||||
Object fromConnectorConnectionConfig;
|
||||
Object toConnectorConnectionConfig;
|
||||
Object fromConnectorLinkConfig;
|
||||
Object toConnectorLinkConfig;
|
||||
Object fromConnectorJobConfig;
|
||||
Object toConnectorJobConfig;
|
||||
Object fromFrameworkConnectionConfig;
|
||||
Object toFrameworkConnectionConfig;
|
||||
Object configFrameworkJob;
|
||||
Object fromFrameworkLinkConfig;
|
||||
Object toFrameworkLinkConfig;
|
||||
Object frameworkJobConfig;
|
||||
|
||||
/**
|
||||
* Connector context (submission specific configuration)
|
||||
|
@ -93,7 +93,7 @@ public class JobRequest {
|
|||
/**
|
||||
* Framework context (submission specific configuration)
|
||||
*/
|
||||
MutableMapContext frameworkContext;
|
||||
MutableMapContext driverContext;
|
||||
|
||||
/**
|
||||
* Optional notification URL for job progress
|
||||
|
@ -119,15 +119,15 @@ public class JobRequest {
|
|||
this.jars = new LinkedList<String>();
|
||||
this.fromConnectorContext = new MutableMapContext();
|
||||
this.toConnectorContext = new MutableMapContext();
|
||||
this.frameworkContext = new MutableMapContext();
|
||||
this.driverContext = new MutableMapContext();
|
||||
this.fromConnector = null;
|
||||
this.toConnector = null;
|
||||
this.fromConnectorConnectionConfig = null;
|
||||
this.toConnectorConnectionConfig = null;
|
||||
this.fromConnectorLinkConfig = null;
|
||||
this.toConnectorLinkConfig = null;
|
||||
this.fromConnectorJobConfig = null;
|
||||
this.toConnectorJobConfig = null;
|
||||
this.fromFrameworkConnectionConfig = null;
|
||||
this.toFrameworkConnectionConfig = null;
|
||||
this.fromFrameworkLinkConfig = null;
|
||||
this.toFrameworkLinkConfig = null;
|
||||
}
|
||||
|
||||
public MSubmission getSummary() {
|
||||
|
@ -218,26 +218,26 @@ public class JobRequest {
|
|||
this.to = to;
|
||||
}
|
||||
|
||||
public Object getConnectorConnectionConfig(Direction type) {
|
||||
public Object getConnectorLinkConfig(Direction type) {
|
||||
switch(type) {
|
||||
case FROM:
|
||||
return fromConnectorConnectionConfig;
|
||||
return fromConnectorLinkConfig;
|
||||
|
||||
case TO:
|
||||
return toConnectorConnectionConfig;
|
||||
return toConnectorLinkConfig;
|
||||
|
||||
default:
|
||||
throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
public void setConnectorConnectionConfig(Direction type, Object config) {
|
||||
public void setConnectorLinkConfig(Direction type, Object config) {
|
||||
switch(type) {
|
||||
case FROM:
|
||||
fromConnectorConnectionConfig = config;
|
||||
fromConnectorLinkConfig = config;
|
||||
break;
|
||||
case TO:
|
||||
toConnectorConnectionConfig = config;
|
||||
toConnectorLinkConfig = config;
|
||||
break;
|
||||
default:
|
||||
throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
|
||||
|
@ -270,38 +270,38 @@ public class JobRequest {
|
|||
}
|
||||
}
|
||||
|
||||
public Object getFrameworkConnectionConfig(Direction type) {
|
||||
public Object getFrameworkLinkConfig(Direction type) {
|
||||
switch(type) {
|
||||
case FROM:
|
||||
return fromFrameworkConnectionConfig;
|
||||
return fromFrameworkLinkConfig;
|
||||
|
||||
case TO:
|
||||
return toFrameworkConnectionConfig;
|
||||
return toFrameworkLinkConfig;
|
||||
|
||||
default:
|
||||
throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
public void setFrameworkConnectionConfig(Direction type, Object config) {
|
||||
public void setFrameworkLinkConfig(Direction type, Object config) {
|
||||
switch(type) {
|
||||
case FROM:
|
||||
fromFrameworkConnectionConfig = config;
|
||||
fromFrameworkLinkConfig = config;
|
||||
break;
|
||||
case TO:
|
||||
toFrameworkConnectionConfig = config;
|
||||
toFrameworkLinkConfig = config;
|
||||
break;
|
||||
default:
|
||||
throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
public Object getConfigFrameworkJob() {
|
||||
return configFrameworkJob;
|
||||
public Object getFrameworkJobConfig() {
|
||||
return frameworkJobConfig;
|
||||
}
|
||||
|
||||
public void setConfigFrameworkJob(Object config) {
|
||||
configFrameworkJob = config;
|
||||
public void setFrameworkJobConfig(Object config) {
|
||||
frameworkJobConfig = config;
|
||||
}
|
||||
|
||||
public MutableMapContext getConnectorContext(Direction type) {
|
||||
|
@ -317,8 +317,8 @@ public class JobRequest {
|
|||
}
|
||||
}
|
||||
|
||||
public MutableMapContext getFrameworkContext() {
|
||||
return frameworkContext;
|
||||
public MutableMapContext getDriverContext() {
|
||||
return driverContext;
|
||||
}
|
||||
|
||||
public String getNotificationUrl() {
|
||||
|
@ -353,4 +353,4 @@ public class JobRequest {
|
|||
this.intermediateDataFormat = intermediateDataFormat;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -15,7 +15,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework;
|
||||
package org.apache.sqoop.driver;
|
||||
|
||||
import org.apache.sqoop.common.MapContext;
|
||||
import org.apache.sqoop.submission.counter.Counters;
|
||||
|
@ -49,11 +49,11 @@ public abstract class SubmissionEngine {
|
|||
* @param executionEngineClass Configured execution class.
|
||||
* @return True if such execution engine is supported
|
||||
*/
|
||||
public abstract boolean isExecutionEngineSupported(Class executionEngineClass);
|
||||
public abstract boolean isExecutionEngineSupported(Class<?> executionEngineClass);
|
||||
|
||||
/**
|
||||
* Submit new job to remote (hadoop) cluster. This method *must* fill
|
||||
* submission.getSummary.setExternalId(), otherwise Sqoop framework won't
|
||||
* submission.getSummary.setExternalId(), otherwise Sqoop won't
|
||||
* be able to track progress on this job!
|
||||
*
|
||||
* @return Return true if we were able to submit job to remote cluster.
|
||||
|
@ -92,7 +92,7 @@ public abstract class SubmissionEngine {
|
|||
/**
|
||||
* Return statistics for given submission id.
|
||||
*
|
||||
* Sqoop framework will call counters only for submission in state SUCCEEDED,
|
||||
* Sqoop will call counters only for submission in state SUCCEEDED,
|
||||
* it's consider exceptional state to call this method for other states.
|
||||
*
|
||||
* @param submissionId Submission internal id.
|
|
@ -15,11 +15,14 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework.configuration;
|
||||
package org.apache.sqoop.driver.configuration;
|
||||
|
||||
import org.apache.sqoop.model.ConfigurationClass;
|
||||
import org.apache.sqoop.model.Form;
|
||||
|
||||
/**
|
||||
* Representing the core job configuration
|
||||
*/
|
||||
@ConfigurationClass
|
||||
public class JobConfiguration {
|
||||
@Form
|
|
@ -15,16 +15,14 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.sqoop.framework.configuration;
|
||||
package org.apache.sqoop.driver.configuration;
|
||||
|
||||
import org.apache.sqoop.model.ConfigurationClass;
|
||||
import org.apache.sqoop.model.Form;
|
||||
|
||||
/**
|
||||
* Framework class representing connection configuration
|
||||
* Representing the core link configuration
|
||||
*/
|
||||
@ConfigurationClass
|
||||
public class ConnectionConfiguration {
|
||||
public class LinkConfiguration {
|
||||
|
||||
@Form SecurityForm security = new SecurityForm();
|
||||
}
|
Некоторые файлы не были показаны из-за слишком большого количества измененных файлов Показать больше
Загрузка…
Ссылка в новой задаче