mirror of
https://github.com/apache/sqoop.git
synced 2025-05-04 05:39:35 +08:00
SQOOP-1417: Sqoop2: From/To: Change ConnectorType name
(Abraham Elmahrek via Jarek Jarcec Cecho)
This commit is contained in:
parent
88eb766e56
commit
a41ce9e511
@ -18,7 +18,7 @@
|
|||||||
package org.apache.sqoop.client;
|
package org.apache.sqoop.client;
|
||||||
|
|
||||||
import org.apache.sqoop.client.request.SqoopRequests;
|
import org.apache.sqoop.client.request.SqoopRequests;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.json.ConnectionValidationBean;
|
import org.apache.sqoop.json.ConnectionValidationBean;
|
||||||
import org.apache.sqoop.json.ConnectorBean;
|
import org.apache.sqoop.json.ConnectorBean;
|
||||||
@ -368,8 +368,8 @@ public MJob newJob(long fromXid, long toXid) {
|
|||||||
toConnection.getConnectorId(),
|
toConnection.getConnectorId(),
|
||||||
fromConnection.getPersistenceId(),
|
fromConnection.getPersistenceId(),
|
||||||
toConnection.getPersistenceId(),
|
toConnection.getPersistenceId(),
|
||||||
getConnector(fromConnection.getConnectorId()).getJobForms(ConnectorType.FROM),
|
getConnector(fromConnection.getConnectorId()).getJobForms(Direction.FROM),
|
||||||
getConnector(fromConnection.getConnectorId()).getJobForms(ConnectorType.TO),
|
getConnector(fromConnection.getConnectorId()).getJobForms(Direction.TO),
|
||||||
getFramework().getJobForms()
|
getFramework().getJobForms()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -550,17 +550,17 @@ private Status applyValidations(ConnectionValidationBean bean, MConnection conne
|
|||||||
}
|
}
|
||||||
|
|
||||||
private Status applyValidations(JobValidationBean bean, MJob job) {
|
private Status applyValidations(JobValidationBean bean, MJob job) {
|
||||||
Validation fromConnector = bean.getConnectorValidation(ConnectorType.FROM);
|
Validation fromConnector = bean.getConnectorValidation(Direction.FROM);
|
||||||
Validation toConnector = bean.getConnectorValidation(ConnectorType.TO);
|
Validation toConnector = bean.getConnectorValidation(Direction.TO);
|
||||||
Validation framework = bean.getFrameworkValidation();
|
Validation framework = bean.getFrameworkValidation();
|
||||||
|
|
||||||
// @TODO(Abe): From/To validation.
|
// @TODO(Abe): From/To validation.
|
||||||
FormUtils.applyValidation(
|
FormUtils.applyValidation(
|
||||||
job.getConnectorPart(ConnectorType.FROM).getForms(),
|
job.getConnectorPart(Direction.FROM).getForms(),
|
||||||
fromConnector);
|
fromConnector);
|
||||||
FormUtils.applyValidation(job.getFrameworkPart().getForms(), framework);
|
FormUtils.applyValidation(job.getFrameworkPart().getForms(), framework);
|
||||||
FormUtils.applyValidation(
|
FormUtils.applyValidation(
|
||||||
job.getConnectorPart(ConnectorType.TO).getForms(),
|
job.getConnectorPart(Direction.TO).getForms(),
|
||||||
toConnector);
|
toConnector);
|
||||||
|
|
||||||
Long id = bean.getId();
|
Long id = bean.getId();
|
||||||
|
@ -24,7 +24,7 @@
|
|||||||
* is being used to load data TO, then the connector type
|
* is being used to load data TO, then the connector type
|
||||||
* will be TO.
|
* will be TO.
|
||||||
*/
|
*/
|
||||||
public enum ConnectorType {
|
public enum Direction {
|
||||||
FROM,
|
FROM,
|
||||||
TO
|
TO
|
||||||
}
|
}
|
@ -17,7 +17,7 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.sqoop.common;
|
package org.apache.sqoop.common;
|
||||||
|
|
||||||
public enum ConnectorTypeError implements ErrorCode {
|
public enum DirectionError implements ErrorCode {
|
||||||
|
|
||||||
/** An unknown error has occurred. */
|
/** An unknown error has occurred. */
|
||||||
CONNECTOR_TYPE_0000("Unknown connector type")
|
CONNECTOR_TYPE_0000("Unknown connector type")
|
||||||
@ -26,7 +26,7 @@ public enum ConnectorTypeError implements ErrorCode {
|
|||||||
|
|
||||||
private final String message;
|
private final String message;
|
||||||
|
|
||||||
private ConnectorTypeError(String message) {
|
private DirectionError(String message) {
|
||||||
this.message = message;
|
this.message = message;
|
||||||
}
|
}
|
||||||
|
|
@ -24,7 +24,7 @@
|
|||||||
import java.util.ResourceBundle;
|
import java.util.ResourceBundle;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MConnectionForms;
|
import org.apache.sqoop.model.MConnectionForms;
|
||||||
import org.apache.sqoop.model.MJobForms;
|
import org.apache.sqoop.model.MJobForms;
|
||||||
import org.apache.sqoop.model.MConnector;
|
import org.apache.sqoop.model.MConnector;
|
||||||
@ -77,9 +77,9 @@ public JSONObject extract(boolean skipSensitive) {
|
|||||||
object.put(CON_FORMS, extractForms(connector.getConnectionForms().getForms(), skipSensitive));
|
object.put(CON_FORMS, extractForms(connector.getConnectionForms().getForms(), skipSensitive));
|
||||||
object.put(JOB_FORMS, new JSONObject());
|
object.put(JOB_FORMS, new JSONObject());
|
||||||
((JSONObject)object.get(JOB_FORMS)).put(
|
((JSONObject)object.get(JOB_FORMS)).put(
|
||||||
ConnectorType.FROM, extractForms(connector.getJobForms(ConnectorType.FROM).getForms(), skipSensitive));
|
Direction.FROM, extractForms(connector.getJobForms(Direction.FROM).getForms(), skipSensitive));
|
||||||
((JSONObject)object.get(JOB_FORMS)).put(
|
((JSONObject)object.get(JOB_FORMS)).put(
|
||||||
ConnectorType.TO, extractForms(connector.getJobForms(ConnectorType.TO).getForms(), skipSensitive));
|
Direction.TO, extractForms(connector.getJobForms(Direction.TO).getForms(), skipSensitive));
|
||||||
array.add(object);
|
array.add(object);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -118,8 +118,8 @@ public void restore(JSONObject jsonObject) {
|
|||||||
List<MForm> connForms = restoreForms((JSONArray) object.get(CON_FORMS));
|
List<MForm> connForms = restoreForms((JSONArray) object.get(CON_FORMS));
|
||||||
|
|
||||||
JSONObject jobJson = (JSONObject) object.get(JOB_FORMS);
|
JSONObject jobJson = (JSONObject) object.get(JOB_FORMS);
|
||||||
JSONArray fromJobJson = (JSONArray)jobJson.get(ConnectorType.FROM.name());
|
JSONArray fromJobJson = (JSONArray)jobJson.get(Direction.FROM.name());
|
||||||
JSONArray toJobJson = (JSONArray)jobJson.get(ConnectorType.TO.name());
|
JSONArray toJobJson = (JSONArray)jobJson.get(Direction.TO.name());
|
||||||
List<MForm> fromJobForms =
|
List<MForm> fromJobForms =
|
||||||
restoreForms(fromJobJson);
|
restoreForms(fromJobJson);
|
||||||
List<MForm> toJobForms =
|
List<MForm> toJobForms =
|
||||||
|
@ -17,7 +17,7 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.sqoop.json;
|
package org.apache.sqoop.json;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MForm;
|
import org.apache.sqoop.model.MForm;
|
||||||
import org.apache.sqoop.model.MJob;
|
import org.apache.sqoop.model.MJob;
|
||||||
import org.apache.sqoop.model.MJobForms;
|
import org.apache.sqoop.model.MJobForms;
|
||||||
@ -114,14 +114,14 @@ public JSONObject extract(boolean skipSensitive) {
|
|||||||
object.put(CREATION_DATE, job.getCreationDate().getTime());
|
object.put(CREATION_DATE, job.getCreationDate().getTime());
|
||||||
object.put(UPDATE_USER, job.getLastUpdateUser());
|
object.put(UPDATE_USER, job.getLastUpdateUser());
|
||||||
object.put(UPDATE_DATE, job.getLastUpdateDate().getTime());
|
object.put(UPDATE_DATE, job.getLastUpdateDate().getTime());
|
||||||
object.put(FROM_CONNECTION_ID, job.getConnectionId(ConnectorType.FROM));
|
object.put(FROM_CONNECTION_ID, job.getConnectionId(Direction.FROM));
|
||||||
object.put(TO_CONNECTION_ID, job.getConnectionId(ConnectorType.TO));
|
object.put(TO_CONNECTION_ID, job.getConnectionId(Direction.TO));
|
||||||
object.put(FROM_CONNECTOR_ID, job.getConnectorId(ConnectorType.FROM));
|
object.put(FROM_CONNECTOR_ID, job.getConnectorId(Direction.FROM));
|
||||||
object.put(TO_CONNECTOR_ID, job.getConnectorId(ConnectorType.TO));
|
object.put(TO_CONNECTOR_ID, job.getConnectorId(Direction.TO));
|
||||||
object.put(FROM_CONNECTOR_PART,
|
object.put(FROM_CONNECTOR_PART,
|
||||||
extractForms(job.getConnectorPart(ConnectorType.FROM).getForms(),skipSensitive));
|
extractForms(job.getConnectorPart(Direction.FROM).getForms(),skipSensitive));
|
||||||
object.put(TO_CONNECTOR_PART,
|
object.put(TO_CONNECTOR_PART,
|
||||||
extractForms(job.getConnectorPart(ConnectorType.TO).getForms(), skipSensitive));
|
extractForms(job.getConnectorPart(Direction.TO).getForms(), skipSensitive));
|
||||||
object.put(FRAMEWORK_PART,
|
object.put(FRAMEWORK_PART,
|
||||||
extractForms(job.getFrameworkPart().getForms(), skipSensitive));
|
extractForms(job.getFrameworkPart().getForms(), skipSensitive));
|
||||||
|
|
||||||
|
@ -17,8 +17,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.sqoop.json;
|
package org.apache.sqoop.json;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.ConnectorTypeError;
|
import org.apache.sqoop.common.DirectionError;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.validation.Status;
|
import org.apache.sqoop.validation.Status;
|
||||||
import org.apache.sqoop.validation.Validation;
|
import org.apache.sqoop.validation.Validation;
|
||||||
@ -63,7 +63,7 @@ public JobValidationBean() {
|
|||||||
id = null;
|
id = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Validation getConnectorValidation(ConnectorType type) {
|
public Validation getConnectorValidation(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectorValidation;
|
return fromConnectorValidation;
|
||||||
@ -72,7 +72,7 @@ public Validation getConnectorValidation(ConnectorType type) {
|
|||||||
return toConnectorValidation;
|
return toConnectorValidation;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -98,8 +98,8 @@ public JSONObject extract(boolean skipSensitive) {
|
|||||||
object.put(ID, id);
|
object.put(ID, id);
|
||||||
}
|
}
|
||||||
|
|
||||||
connectorObject.put(FROM, extractValidation(getConnectorValidation(ConnectorType.FROM)));
|
connectorObject.put(FROM, extractValidation(getConnectorValidation(Direction.FROM)));
|
||||||
connectorObject.put(TO, extractValidation(getConnectorValidation(ConnectorType.TO)));
|
connectorObject.put(TO, extractValidation(getConnectorValidation(Direction.TO)));
|
||||||
|
|
||||||
object.put(FRAMEWORK, extractValidation(frameworkValidation));
|
object.put(FRAMEWORK, extractValidation(frameworkValidation));
|
||||||
object.put(CONNECTOR, connectorObject);
|
object.put(CONNECTOR, connectorObject);
|
||||||
|
@ -17,8 +17,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.sqoop.model;
|
package org.apache.sqoop.model;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.ConnectorTypeError;
|
import org.apache.sqoop.common.DirectionError;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -66,8 +66,8 @@ public String toString() {
|
|||||||
sb.append(uniqueName).append(":").append(getPersistenceId()).append(":");
|
sb.append(uniqueName).append(":").append(getPersistenceId()).append(":");
|
||||||
sb.append(className);
|
sb.append(className);
|
||||||
sb.append(", ").append(getConnectionForms().toString());
|
sb.append(", ").append(getConnectionForms().toString());
|
||||||
sb.append(", ").append(getJobForms(ConnectorType.FROM).toString());
|
sb.append(", ").append(getJobForms(Direction.FROM).toString());
|
||||||
sb.append(", ").append(getJobForms(ConnectorType.TO).toString());
|
sb.append(", ").append(getJobForms(Direction.TO).toString());
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -86,15 +86,15 @@ public boolean equals(Object other) {
|
|||||||
&& className.equals(mc.className)
|
&& className.equals(mc.className)
|
||||||
&& version.equals(mc.version)
|
&& version.equals(mc.version)
|
||||||
&& connectionForms.equals(mc.getConnectionForms())
|
&& connectionForms.equals(mc.getConnectionForms())
|
||||||
&& fromJobForms.equals(mc.getJobForms(ConnectorType.FROM))
|
&& fromJobForms.equals(mc.getJobForms(Direction.FROM))
|
||||||
&& toJobForms.equals(mc.getJobForms(ConnectorType.TO));
|
&& toJobForms.equals(mc.getJobForms(Direction.TO));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
int result = getConnectionForms().hashCode();
|
int result = getConnectionForms().hashCode();
|
||||||
result = 31 * result + getJobForms(ConnectorType.FROM).hashCode();
|
result = 31 * result + getJobForms(Direction.FROM).hashCode();
|
||||||
result = 31 * result + getJobForms(ConnectorType.TO).hashCode();
|
result = 31 * result + getJobForms(Direction.TO).hashCode();
|
||||||
result = 31 * result + version.hashCode();
|
result = 31 * result + version.hashCode();
|
||||||
result = 31 * result + uniqueName.hashCode();
|
result = 31 * result + uniqueName.hashCode();
|
||||||
result = 31 * result + className.hashCode();
|
result = 31 * result + className.hashCode();
|
||||||
@ -109,8 +109,8 @@ public MConnector clone(boolean cloneWithValue) {
|
|||||||
this.getClassName(),
|
this.getClassName(),
|
||||||
this.getVersion(),
|
this.getVersion(),
|
||||||
this.getConnectionForms().clone(cloneWithValue),
|
this.getConnectionForms().clone(cloneWithValue),
|
||||||
this.getJobForms(ConnectorType.FROM).clone(cloneWithValue),
|
this.getJobForms(Direction.FROM).clone(cloneWithValue),
|
||||||
this.getJobForms(ConnectorType.TO).clone(cloneWithValue));
|
this.getJobForms(Direction.TO).clone(cloneWithValue));
|
||||||
copy.setPersistenceId(this.getPersistenceId());
|
copy.setPersistenceId(this.getPersistenceId());
|
||||||
return copy;
|
return copy;
|
||||||
}
|
}
|
||||||
@ -119,7 +119,7 @@ public MConnectionForms getConnectionForms() {
|
|||||||
return connectionForms;
|
return connectionForms;
|
||||||
}
|
}
|
||||||
|
|
||||||
public MJobForms getJobForms(ConnectorType type) {
|
public MJobForms getJobForms(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromJobForms;
|
return fromJobForms;
|
||||||
@ -128,7 +128,7 @@ public MJobForms getJobForms(ConnectorType type) {
|
|||||||
return toJobForms;
|
return toJobForms;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -17,8 +17,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.sqoop.model;
|
package org.apache.sqoop.model;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.ConnectorTypeError;
|
import org.apache.sqoop.common.DirectionError;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -85,8 +85,8 @@ public MJob(long fromConnectorId,
|
|||||||
*/
|
*/
|
||||||
public MJob(MJob other) {
|
public MJob(MJob other) {
|
||||||
this(other,
|
this(other,
|
||||||
other.getConnectorPart(ConnectorType.FROM).clone(true),
|
other.getConnectorPart(Direction.FROM).clone(true),
|
||||||
other.getConnectorPart(ConnectorType.TO).clone(true),
|
other.getConnectorPart(Direction.TO).clone(true),
|
||||||
other.frameworkPart.clone(true));
|
other.frameworkPart.clone(true));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -106,10 +106,10 @@ public MJob(MJob other, MJobForms fromPart, MJobForms toPart, MJobForms framewor
|
|||||||
|
|
||||||
this.name = other.name;
|
this.name = other.name;
|
||||||
|
|
||||||
this.fromConnectorId = other.getConnectorId(ConnectorType.FROM);
|
this.fromConnectorId = other.getConnectorId(Direction.FROM);
|
||||||
this.toConnectorId = other.getConnectorId(ConnectorType.TO);
|
this.toConnectorId = other.getConnectorId(Direction.TO);
|
||||||
this.fromConnectionId = other.getConnectionId(ConnectorType.FROM);
|
this.fromConnectionId = other.getConnectionId(Direction.FROM);
|
||||||
this.toConnectionId = other.getConnectionId(ConnectorType.TO);
|
this.toConnectionId = other.getConnectionId(Direction.TO);
|
||||||
this.fromConnectorPart = fromPart;
|
this.fromConnectorPart = fromPart;
|
||||||
this.toConnectorPart = toPart;
|
this.toConnectorPart = toPart;
|
||||||
this.frameworkPart = frameworkPart;
|
this.frameworkPart = frameworkPart;
|
||||||
@ -118,8 +118,8 @@ public MJob(MJob other, MJobForms fromPart, MJobForms toPart, MJobForms framewor
|
|||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
StringBuilder sb = new StringBuilder("job");
|
StringBuilder sb = new StringBuilder("job");
|
||||||
sb.append(" connector-from-part: ").append(getConnectorPart(ConnectorType.FROM));
|
sb.append(" connector-from-part: ").append(getConnectorPart(Direction.FROM));
|
||||||
sb.append(", connector-to-part: ").append(getConnectorPart(ConnectorType.TO));
|
sb.append(", connector-to-part: ").append(getConnectorPart(Direction.TO));
|
||||||
sb.append(", framework-part: ").append(frameworkPart);
|
sb.append(", framework-part: ").append(frameworkPart);
|
||||||
|
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
@ -133,7 +133,7 @@ public void setName(String name) {
|
|||||||
this.name = name;
|
this.name = name;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getConnectionId(ConnectorType type) {
|
public long getConnectionId(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectionId;
|
return fromConnectionId;
|
||||||
@ -142,11 +142,11 @@ public long getConnectionId(ConnectorType type) {
|
|||||||
return toConnectionId;
|
return toConnectionId;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getConnectorId(ConnectorType type) {
|
public long getConnectorId(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectorId;
|
return fromConnectorId;
|
||||||
@ -155,11 +155,11 @@ public long getConnectorId(ConnectorType type) {
|
|||||||
return toConnectorId;
|
return toConnectorId;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public MJobForms getConnectorPart(ConnectorType type) {
|
public MJobForms getConnectorPart(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectorPart;
|
return fromConnectorPart;
|
||||||
@ -168,7 +168,7 @@ public MJobForms getConnectorPart(ConnectorType type) {
|
|||||||
return toConnectorPart;
|
return toConnectorPart;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -182,12 +182,12 @@ public MJob clone(boolean cloneWithValue) {
|
|||||||
return new MJob(this);
|
return new MJob(this);
|
||||||
} else {
|
} else {
|
||||||
return new MJob(
|
return new MJob(
|
||||||
getConnectorId(ConnectorType.FROM),
|
getConnectorId(Direction.FROM),
|
||||||
getConnectorId(ConnectorType.TO),
|
getConnectorId(Direction.TO),
|
||||||
getConnectionId(ConnectorType.FROM),
|
getConnectionId(Direction.FROM),
|
||||||
getConnectionId(ConnectorType.TO),
|
getConnectionId(Direction.TO),
|
||||||
getConnectorPart(ConnectorType.FROM).clone(false),
|
getConnectorPart(Direction.FROM).clone(false),
|
||||||
getConnectorPart(ConnectorType.TO).clone(false),
|
getConnectorPart(Direction.TO).clone(false),
|
||||||
frameworkPart.clone(false));
|
frameworkPart.clone(false));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -203,13 +203,13 @@ public boolean equals(Object object) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
MJob job = (MJob)object;
|
MJob job = (MJob)object;
|
||||||
return (job.getConnectorId(ConnectorType.FROM) == this.getConnectorId(ConnectorType.FROM))
|
return (job.getConnectorId(Direction.FROM) == this.getConnectorId(Direction.FROM))
|
||||||
&& (job.getConnectorId(ConnectorType.TO) == this.getConnectorId(ConnectorType.TO))
|
&& (job.getConnectorId(Direction.TO) == this.getConnectorId(Direction.TO))
|
||||||
&& (job.getConnectionId(ConnectorType.FROM) == this.getConnectionId(ConnectorType.FROM))
|
&& (job.getConnectionId(Direction.FROM) == this.getConnectionId(Direction.FROM))
|
||||||
&& (job.getConnectionId(ConnectorType.TO) == this.getConnectionId(ConnectorType.TO))
|
&& (job.getConnectionId(Direction.TO) == this.getConnectionId(Direction.TO))
|
||||||
&& (job.getPersistenceId() == this.getPersistenceId())
|
&& (job.getPersistenceId() == this.getPersistenceId())
|
||||||
&& (job.getConnectorPart(ConnectorType.FROM).equals(this.getConnectorPart(ConnectorType.FROM)))
|
&& (job.getConnectorPart(Direction.FROM).equals(this.getConnectorPart(Direction.FROM)))
|
||||||
&& (job.getConnectorPart(ConnectorType.TO).equals(this.getConnectorPart(ConnectorType.TO)))
|
&& (job.getConnectorPart(Direction.TO).equals(this.getConnectorPart(Direction.TO)))
|
||||||
&& (job.frameworkPart.equals(this.frameworkPart));
|
&& (job.frameworkPart.equals(this.frameworkPart));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,10 +20,8 @@
|
|||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.ResourceBundle;
|
import java.util.ResourceBundle;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.VersionInfo;
|
import org.apache.sqoop.common.VersionInfo;
|
||||||
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
|
|
||||||
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
|
||||||
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
|
||||||
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
|
||||||
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
|
||||||
@ -72,7 +70,7 @@ public Class getConnectionConfigurationClass() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Class getJobConfigurationClass(ConnectorType jobType) {
|
public Class getJobConfigurationClass(Direction jobType) {
|
||||||
switch (jobType) {
|
switch (jobType) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return FromJobConfiguration.class;
|
return FromJobConfiguration.class;
|
||||||
|
@ -22,7 +22,7 @@
|
|||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.core.ConfigurationConstants;
|
import org.apache.sqoop.core.ConfigurationConstants;
|
||||||
import org.apache.sqoop.model.FormUtils;
|
import org.apache.sqoop.model.FormUtils;
|
||||||
import org.apache.sqoop.model.MConnectionForms;
|
import org.apache.sqoop.model.MConnectionForms;
|
||||||
@ -92,11 +92,11 @@ public ConnectorHandler(URL configFileUrl) {
|
|||||||
|
|
||||||
// Initialize Metadata
|
// Initialize Metadata
|
||||||
MJobForms fromJobForms = new MJobForms(FormUtils.toForms(
|
MJobForms fromJobForms = new MJobForms(FormUtils.toForms(
|
||||||
connector.getJobConfigurationClass(ConnectorType.FROM)));
|
connector.getJobConfigurationClass(Direction.FROM)));
|
||||||
MConnectionForms connectionForms = new MConnectionForms(
|
MConnectionForms connectionForms = new MConnectionForms(
|
||||||
FormUtils.toForms(connector.getConnectionConfigurationClass()));
|
FormUtils.toForms(connector.getConnectionConfigurationClass()));
|
||||||
MJobForms toJobForms = new MJobForms(FormUtils.toForms(
|
MJobForms toJobForms = new MJobForms(FormUtils.toForms(
|
||||||
connector.getJobConfigurationClass(ConnectorType.TO)));
|
connector.getJobConfigurationClass(Direction.TO)));
|
||||||
MConnectionForms toConnectionForms = new MConnectionForms(
|
MConnectionForms toConnectionForms = new MConnectionForms(
|
||||||
FormUtils.toForms(connector.getConnectionConfigurationClass()));
|
FormUtils.toForms(connector.getConnectionConfigurationClass()));
|
||||||
|
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
package org.apache.sqoop.framework;
|
package org.apache.sqoop.framework;
|
||||||
|
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.MapContext;
|
import org.apache.sqoop.common.MapContext;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.connector.ConnectorManager;
|
import org.apache.sqoop.connector.ConnectorManager;
|
||||||
@ -280,8 +280,8 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
"Job id: " + job.getPersistenceId());
|
"Job id: " + job.getPersistenceId());
|
||||||
}
|
}
|
||||||
|
|
||||||
MConnection fromConnection = repository.findConnection(job.getConnectionId(ConnectorType.FROM));
|
MConnection fromConnection = repository.findConnection(job.getConnectionId(Direction.FROM));
|
||||||
MConnection toConnection = repository.findConnection(job.getConnectionId(ConnectorType.TO));
|
MConnection toConnection = repository.findConnection(job.getConnectionId(Direction.TO));
|
||||||
|
|
||||||
if (!fromConnection.getEnabled()) {
|
if (!fromConnection.getEnabled()) {
|
||||||
throw new SqoopException(FrameworkError.FRAMEWORK_0010,
|
throw new SqoopException(FrameworkError.FRAMEWORK_0010,
|
||||||
@ -294,9 +294,9 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
SqoopConnector fromConnector =
|
SqoopConnector fromConnector =
|
||||||
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.FROM));
|
ConnectorManager.getInstance().getConnector(job.getConnectorId(Direction.FROM));
|
||||||
SqoopConnector toConnector =
|
SqoopConnector toConnector =
|
||||||
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.TO));
|
ConnectorManager.getInstance().getConnector(job.getConnectorId(Direction.TO));
|
||||||
|
|
||||||
// Transform forms to fromConnector specific classes
|
// Transform forms to fromConnector specific classes
|
||||||
Object fromConnectorConnection = ClassUtils.instantiate(
|
Object fromConnectorConnection = ClassUtils.instantiate(
|
||||||
@ -305,9 +305,9 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
fromConnectorConnection);
|
fromConnectorConnection);
|
||||||
|
|
||||||
Object fromJob = ClassUtils.instantiate(
|
Object fromJob = ClassUtils.instantiate(
|
||||||
fromConnector.getJobConfigurationClass(ConnectorType.FROM));
|
fromConnector.getJobConfigurationClass(Direction.FROM));
|
||||||
FormUtils.fromForms(
|
FormUtils.fromForms(
|
||||||
job.getConnectorPart(ConnectorType.FROM).getForms(), fromJob);
|
job.getConnectorPart(Direction.FROM).getForms(), fromJob);
|
||||||
|
|
||||||
// Transform forms to toConnector specific classes
|
// Transform forms to toConnector specific classes
|
||||||
Object toConnectorConnection = ClassUtils.instantiate(
|
Object toConnectorConnection = ClassUtils.instantiate(
|
||||||
@ -316,8 +316,8 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
toConnectorConnection);
|
toConnectorConnection);
|
||||||
|
|
||||||
Object toJob = ClassUtils.instantiate(
|
Object toJob = ClassUtils.instantiate(
|
||||||
toConnector.getJobConfigurationClass(ConnectorType.TO));
|
toConnector.getJobConfigurationClass(Direction.TO));
|
||||||
FormUtils.fromForms(job.getConnectorPart(ConnectorType.TO).getForms(), toJob);
|
FormUtils.fromForms(job.getConnectorPart(Direction.TO).getForms(), toJob);
|
||||||
|
|
||||||
// Transform framework specific forms
|
// Transform framework specific forms
|
||||||
Object fromFrameworkConnection = ClassUtils.instantiate(
|
Object fromFrameworkConnection = ClassUtils.instantiate(
|
||||||
@ -342,15 +342,15 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
|
|
||||||
// Save important variables to the submission request
|
// Save important variables to the submission request
|
||||||
request.setSummary(summary);
|
request.setSummary(summary);
|
||||||
request.setConnector(ConnectorType.FROM, fromConnector);
|
request.setConnector(Direction.FROM, fromConnector);
|
||||||
request.setConnector(ConnectorType.TO, toConnector);
|
request.setConnector(Direction.TO, toConnector);
|
||||||
request.setConnectorConnectionConfig(ConnectorType.FROM, fromConnectorConnection);
|
request.setConnectorConnectionConfig(Direction.FROM, fromConnectorConnection);
|
||||||
request.setConnectorConnectionConfig(ConnectorType.TO, toConnectorConnection);
|
request.setConnectorConnectionConfig(Direction.TO, toConnectorConnection);
|
||||||
request.setConnectorJobConfig(ConnectorType.FROM, fromJob);
|
request.setConnectorJobConfig(Direction.FROM, fromJob);
|
||||||
request.setConnectorJobConfig(ConnectorType.TO, toJob);
|
request.setConnectorJobConfig(Direction.TO, toJob);
|
||||||
// @TODO(Abe): Should we actually have 2 different Framework Connection config objects?
|
// @TODO(Abe): Should we actually have 2 different Framework Connection config objects?
|
||||||
request.setFrameworkConnectionConfig(ConnectorType.FROM, fromFrameworkConnection);
|
request.setFrameworkConnectionConfig(Direction.FROM, fromFrameworkConnection);
|
||||||
request.setFrameworkConnectionConfig(ConnectorType.TO, toFrameworkConnection);
|
request.setFrameworkConnectionConfig(Direction.TO, toFrameworkConnection);
|
||||||
request.setConfigFrameworkJob(frameworkJob);
|
request.setConfigFrameworkJob(frameworkJob);
|
||||||
request.setJobName(job.getName());
|
request.setJobName(job.getName());
|
||||||
request.setJobId(job.getPersistenceId());
|
request.setJobId(job.getPersistenceId());
|
||||||
@ -410,24 +410,24 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Initializer context
|
// Initializer context
|
||||||
initializerContext = new InitializerContext(request.getConnectorContext(ConnectorType.FROM));
|
initializerContext = new InitializerContext(request.getConnectorContext(Direction.FROM));
|
||||||
|
|
||||||
// Initialize submission from fromConnector perspective
|
// Initialize submission from fromConnector perspective
|
||||||
initializer.initialize(initializerContext,
|
initializer.initialize(initializerContext,
|
||||||
request.getConnectorConnectionConfig(ConnectorType.FROM),
|
request.getConnectorConnectionConfig(Direction.FROM),
|
||||||
request.getConnectorJobConfig(ConnectorType.FROM));
|
request.getConnectorJobConfig(Direction.FROM));
|
||||||
|
|
||||||
// Add job specific jars to
|
// Add job specific jars to
|
||||||
request.addJars(initializer.getJars(initializerContext,
|
request.addJars(initializer.getJars(initializerContext,
|
||||||
request.getConnectorConnectionConfig(ConnectorType.FROM),
|
request.getConnectorConnectionConfig(Direction.FROM),
|
||||||
request.getConnectorJobConfig(ConnectorType.FROM)));
|
request.getConnectorJobConfig(Direction.FROM)));
|
||||||
|
|
||||||
// @TODO(Abe): Alter behavior of Schema here. Need from Schema.
|
// @TODO(Abe): Alter behavior of Schema here. Need from Schema.
|
||||||
// Retrieve and persist the schema
|
// Retrieve and persist the schema
|
||||||
request.getSummary().setConnectorSchema(initializer.getSchema(
|
request.getSummary().setConnectorSchema(initializer.getSchema(
|
||||||
initializerContext,
|
initializerContext,
|
||||||
request.getConnectorConnectionConfig(ConnectorType.FROM),
|
request.getConnectorConnectionConfig(Direction.FROM),
|
||||||
request.getConnectorJobConfig(ConnectorType.FROM)
|
request.getConnectorJobConfig(Direction.FROM)
|
||||||
));
|
));
|
||||||
|
|
||||||
// Initialize To Connector callback.
|
// Initialize To Connector callback.
|
||||||
@ -444,17 +444,17 @@ public MSubmission submit(long jobId, HttpEventContext ctx) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Initializer context
|
// Initializer context
|
||||||
initializerContext = new InitializerContext(request.getConnectorContext(ConnectorType.TO));
|
initializerContext = new InitializerContext(request.getConnectorContext(Direction.TO));
|
||||||
|
|
||||||
// Initialize submission from fromConnector perspective
|
// Initialize submission from fromConnector perspective
|
||||||
initializer.initialize(initializerContext,
|
initializer.initialize(initializerContext,
|
||||||
request.getConnectorConnectionConfig(ConnectorType.TO),
|
request.getConnectorConnectionConfig(Direction.TO),
|
||||||
request.getConnectorJobConfig(ConnectorType.TO));
|
request.getConnectorJobConfig(Direction.TO));
|
||||||
|
|
||||||
// Add job specific jars to
|
// Add job specific jars to
|
||||||
request.addJars(initializer.getJars(initializerContext,
|
request.addJars(initializer.getJars(initializerContext,
|
||||||
request.getConnectorConnectionConfig(ConnectorType.TO),
|
request.getConnectorConnectionConfig(Direction.TO),
|
||||||
request.getConnectorJobConfig(ConnectorType.TO)));
|
request.getConnectorJobConfig(Direction.TO)));
|
||||||
|
|
||||||
// @TODO(Abe): Alter behavior of Schema here. Need To Schema.
|
// @TODO(Abe): Alter behavior of Schema here. Need To Schema.
|
||||||
// Retrieve and persist the schema
|
// Retrieve and persist the schema
|
||||||
@ -531,17 +531,17 @@ private void destroySubmission(SubmissionRequest request) {
|
|||||||
|
|
||||||
// @TODO(Abe): Update context to manage multiple connectors. As well as summary.
|
// @TODO(Abe): Update context to manage multiple connectors. As well as summary.
|
||||||
DestroyerContext fromDestroyerContext = new DestroyerContext(
|
DestroyerContext fromDestroyerContext = new DestroyerContext(
|
||||||
request.getConnectorContext(ConnectorType.FROM), false, request.getSummary()
|
request.getConnectorContext(Direction.FROM), false, request.getSummary()
|
||||||
.getConnectorSchema());
|
.getConnectorSchema());
|
||||||
DestroyerContext toDestroyerContext = new DestroyerContext(
|
DestroyerContext toDestroyerContext = new DestroyerContext(
|
||||||
request.getConnectorContext(ConnectorType.TO), false, request.getSummary()
|
request.getConnectorContext(Direction.TO), false, request.getSummary()
|
||||||
.getConnectorSchema());
|
.getConnectorSchema());
|
||||||
|
|
||||||
// Initialize submission from connector perspective
|
// Initialize submission from connector perspective
|
||||||
fromDestroyer.destroy(fromDestroyerContext, request.getConnectorConnectionConfig(ConnectorType.FROM),
|
fromDestroyer.destroy(fromDestroyerContext, request.getConnectorConnectionConfig(Direction.FROM),
|
||||||
request.getConnectorJobConfig(ConnectorType.FROM));
|
request.getConnectorJobConfig(Direction.FROM));
|
||||||
toDestroyer.destroy(toDestroyerContext, request.getConnectorConnectionConfig(ConnectorType.TO),
|
toDestroyer.destroy(toDestroyerContext, request.getConnectorConnectionConfig(Direction.TO),
|
||||||
request.getConnectorJobConfig(ConnectorType.TO));
|
request.getConnectorJobConfig(Direction.TO));
|
||||||
}
|
}
|
||||||
|
|
||||||
public MSubmission stop(long jobId, HttpEventContext ctx) {
|
public MSubmission stop(long jobId, HttpEventContext ctx) {
|
||||||
|
@ -17,8 +17,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.sqoop.framework;
|
package org.apache.sqoop.framework;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.ConnectorTypeError;
|
import org.apache.sqoop.common.DirectionError;
|
||||||
import org.apache.sqoop.common.MutableMapContext;
|
import org.apache.sqoop.common.MutableMapContext;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
||||||
@ -159,7 +159,7 @@ public void setJobId(long jobId) {
|
|||||||
this.jobId = jobId;
|
this.jobId = jobId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SqoopConnector getConnector(ConnectorType type) {
|
public SqoopConnector getConnector(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnector;
|
return fromConnector;
|
||||||
@ -168,11 +168,11 @@ public SqoopConnector getConnector(ConnectorType type) {
|
|||||||
return toConnector;
|
return toConnector;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setConnector(ConnectorType type, SqoopConnector connector) {
|
public void setConnector(Direction type, SqoopConnector connector) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
fromConnector = connector;
|
fromConnector = connector;
|
||||||
@ -181,7 +181,7 @@ public void setConnector(ConnectorType type, SqoopConnector connector) {
|
|||||||
toConnector = connector;
|
toConnector = connector;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -221,7 +221,7 @@ public void setToCallback(CallbackBase toCallback) {
|
|||||||
this.toCallback = toCallback;
|
this.toCallback = toCallback;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Object getConnectorConnectionConfig(ConnectorType type) {
|
public Object getConnectorConnectionConfig(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectorConnectionConfig;
|
return fromConnectorConnectionConfig;
|
||||||
@ -230,11 +230,11 @@ public Object getConnectorConnectionConfig(ConnectorType type) {
|
|||||||
return toConnectorConnectionConfig;
|
return toConnectorConnectionConfig;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setConnectorConnectionConfig(ConnectorType type, Object config) {
|
public void setConnectorConnectionConfig(Direction type, Object config) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
fromConnectorConnectionConfig = config;
|
fromConnectorConnectionConfig = config;
|
||||||
@ -243,11 +243,11 @@ public void setConnectorConnectionConfig(ConnectorType type, Object config) {
|
|||||||
toConnectorConnectionConfig = config;
|
toConnectorConnectionConfig = config;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Object getConnectorJobConfig(ConnectorType type) {
|
public Object getConnectorJobConfig(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectorJobConfig;
|
return fromConnectorJobConfig;
|
||||||
@ -256,11 +256,11 @@ public Object getConnectorJobConfig(ConnectorType type) {
|
|||||||
return toConnectorJobConfig;
|
return toConnectorJobConfig;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setConnectorJobConfig(ConnectorType type, Object config) {
|
public void setConnectorJobConfig(Direction type, Object config) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
fromConnectorJobConfig = config;
|
fromConnectorJobConfig = config;
|
||||||
@ -269,11 +269,11 @@ public void setConnectorJobConfig(ConnectorType type, Object config) {
|
|||||||
toConnectorJobConfig = config;
|
toConnectorJobConfig = config;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Object getFrameworkConnectionConfig(ConnectorType type) {
|
public Object getFrameworkConnectionConfig(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromFrameworkConnectionConfig;
|
return fromFrameworkConnectionConfig;
|
||||||
@ -282,11 +282,11 @@ public Object getFrameworkConnectionConfig(ConnectorType type) {
|
|||||||
return toFrameworkConnectionConfig;
|
return toFrameworkConnectionConfig;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setFrameworkConnectionConfig(ConnectorType type, Object config) {
|
public void setFrameworkConnectionConfig(Direction type, Object config) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
fromFrameworkConnectionConfig = config;
|
fromFrameworkConnectionConfig = config;
|
||||||
@ -295,7 +295,7 @@ public void setFrameworkConnectionConfig(ConnectorType type, Object config) {
|
|||||||
toFrameworkConnectionConfig = config;
|
toFrameworkConnectionConfig = config;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -307,7 +307,7 @@ public void setConfigFrameworkJob(Object config) {
|
|||||||
configFrameworkJob = config;
|
configFrameworkJob = config;
|
||||||
}
|
}
|
||||||
|
|
||||||
public MutableMapContext getConnectorContext(ConnectorType type) {
|
public MutableMapContext getConnectorContext(Direction type) {
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return fromConnectorContext;
|
return fromConnectorContext;
|
||||||
@ -316,7 +316,7 @@ public MutableMapContext getConnectorContext(ConnectorType type) {
|
|||||||
return toConnectorContext;
|
return toConnectorContext;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
package org.apache.sqoop.repository;
|
package org.apache.sqoop.repository;
|
||||||
|
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.connector.ConnectorManager;
|
import org.apache.sqoop.connector.ConnectorManager;
|
||||||
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
import org.apache.sqoop.connector.spi.MetadataUpgrader;
|
||||||
@ -446,16 +446,16 @@ public final void upgradeConnector(MConnector oldConnector, MConnector newConnec
|
|||||||
// Make a new copy of the forms from the connector,
|
// Make a new copy of the forms from the connector,
|
||||||
// else the values will get set in the forms in the connector for
|
// else the values will get set in the forms in the connector for
|
||||||
// each connection.
|
// each connection.
|
||||||
List<MForm> forms = newConnector.getJobForms(ConnectorType.FROM).clone(false).getForms();
|
List<MForm> forms = newConnector.getJobForms(Direction.FROM).clone(false).getForms();
|
||||||
MJobForms newJobForms = new MJobForms(forms);
|
MJobForms newJobForms = new MJobForms(forms);
|
||||||
upgrader.upgrade(job.getConnectorPart(ConnectorType.FROM), newJobForms);
|
upgrader.upgrade(job.getConnectorPart(Direction.FROM), newJobForms);
|
||||||
// @TODO(Abe): Check From and To
|
// @TODO(Abe): Check From and To
|
||||||
MJob newJob = new MJob(job, newJobForms, newJobForms, job.getFrameworkPart());
|
MJob newJob = new MJob(job, newJobForms, newJobForms, job.getFrameworkPart());
|
||||||
|
|
||||||
// Transform form structures to objects for validations
|
// Transform form structures to objects for validations
|
||||||
// @TODO(Abe): Check From and To
|
// @TODO(Abe): Check From and To
|
||||||
Object newConfigurationObject = ClassUtils.instantiate(connector.getJobConfigurationClass(ConnectorType.FROM));
|
Object newConfigurationObject = ClassUtils.instantiate(connector.getJobConfigurationClass(Direction.FROM));
|
||||||
FormUtils.fromForms(newJob.getConnectorPart(ConnectorType.FROM).getForms(), newConfigurationObject);
|
FormUtils.fromForms(newJob.getConnectorPart(Direction.FROM).getForms(), newConfigurationObject);
|
||||||
|
|
||||||
Validation validation = validator.validateJob(newConfigurationObject);
|
Validation validation = validator.validateJob(newConfigurationObject);
|
||||||
if (validation.getStatus().canProceed()) {
|
if (validation.getStatus().canProceed()) {
|
||||||
@ -536,7 +536,7 @@ public final void upgradeFramework(MFramework framework) {
|
|||||||
List<MForm> forms = framework.getJobForms().clone(false).getForms();
|
List<MForm> forms = framework.getJobForms().clone(false).getForms();
|
||||||
MJobForms newJobForms = new MJobForms(forms);
|
MJobForms newJobForms = new MJobForms(forms);
|
||||||
upgrader.upgrade(job.getFrameworkPart(), newJobForms);
|
upgrader.upgrade(job.getFrameworkPart(), newJobForms);
|
||||||
MJob newJob = new MJob(job, job.getConnectorPart(ConnectorType.FROM), job.getConnectorPart(ConnectorType.TO), newJobForms);
|
MJob newJob = new MJob(job, job.getConnectorPart(Direction.FROM), job.getConnectorPart(Direction.TO), newJobForms);
|
||||||
|
|
||||||
// Transform form structures to objects for validations
|
// Transform form structures to objects for validations
|
||||||
Object newConfigurationObject = ClassUtils.instantiate(FrameworkManager.getInstance().getJobConfigurationClass());
|
Object newConfigurationObject = ClassUtils.instantiate(FrameworkManager.getInstance().getJobConfigurationClass());
|
||||||
|
@ -22,7 +22,7 @@
|
|||||||
import org.apache.hadoop.mapred.JobConf;
|
import org.apache.hadoop.mapred.JobConf;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.apache.log4j.PropertyConfigurator;
|
import org.apache.log4j.PropertyConfigurator;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.job.JobConstants;
|
import org.apache.sqoop.job.JobConstants;
|
||||||
import org.apache.sqoop.json.util.SchemaSerialization;
|
import org.apache.sqoop.json.util.SchemaSerialization;
|
||||||
import org.apache.sqoop.model.FormUtils;
|
import org.apache.sqoop.model.FormUtils;
|
||||||
@ -100,7 +100,7 @@ public final class ConfigurationUtils {
|
|||||||
* @param job MapReduce job object
|
* @param job MapReduce job object
|
||||||
* @param obj Configuration object
|
* @param obj Configuration object
|
||||||
*/
|
*/
|
||||||
public static void setConnectorConnectionConfig(ConnectorType type, Job job, Object obj) {
|
public static void setConnectorConnectionConfig(Direction type, Job job, Object obj) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION, obj.getClass().getName());
|
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION, obj.getClass().getName());
|
||||||
@ -120,7 +120,7 @@ public static void setConnectorConnectionConfig(ConnectorType type, Job job, Obj
|
|||||||
* @param job MapReduce job object
|
* @param job MapReduce job object
|
||||||
* @param obj Configuration object
|
* @param obj Configuration object
|
||||||
*/
|
*/
|
||||||
public static void setConnectorJobConfig(ConnectorType type, Job job, Object obj) {
|
public static void setConnectorJobConfig(Direction type, Job job, Object obj) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB, obj.getClass().getName());
|
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB, obj.getClass().getName());
|
||||||
@ -140,7 +140,7 @@ public static void setConnectorJobConfig(ConnectorType type, Job job, Object obj
|
|||||||
* @param job MapReduce job object
|
* @param job MapReduce job object
|
||||||
* @param obj Configuration object
|
* @param obj Configuration object
|
||||||
*/
|
*/
|
||||||
public static void setFrameworkConnectionConfig(ConnectorType type, Job job, Object obj) {
|
public static void setFrameworkConnectionConfig(Direction type, Job job, Object obj) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION, obj.getClass().getName());
|
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION, obj.getClass().getName());
|
||||||
@ -171,7 +171,7 @@ public static void setFrameworkJobConfig(Job job, Object obj) {
|
|||||||
* @param configuration MapReduce configuration object
|
* @param configuration MapReduce configuration object
|
||||||
* @return Configuration object
|
* @return Configuration object
|
||||||
*/
|
*/
|
||||||
public static Object getConnectorConnectionConfig(ConnectorType type, Configuration configuration) {
|
public static Object getConnectorConnectionConfig(Direction type, Configuration configuration) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION, JOB_CONFIG_FROM_CONNECTOR_CONNECTION_KEY);
|
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION, JOB_CONFIG_FROM_CONNECTOR_CONNECTION_KEY);
|
||||||
@ -189,7 +189,7 @@ public static Object getConnectorConnectionConfig(ConnectorType type, Configurat
|
|||||||
* @param configuration MapReduce configuration object
|
* @param configuration MapReduce configuration object
|
||||||
* @return Configuration object
|
* @return Configuration object
|
||||||
*/
|
*/
|
||||||
public static Object getConnectorJobConfig(ConnectorType type, Configuration configuration) {
|
public static Object getConnectorJobConfig(Direction type, Configuration configuration) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB, JOB_CONFIG_FROM_CONNECTOR_JOB_KEY);
|
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB, JOB_CONFIG_FROM_CONNECTOR_JOB_KEY);
|
||||||
@ -207,7 +207,7 @@ public static Object getConnectorJobConfig(ConnectorType type, Configuration con
|
|||||||
* @param configuration MapReduce configuration object
|
* @param configuration MapReduce configuration object
|
||||||
* @return Configuration object
|
* @return Configuration object
|
||||||
*/
|
*/
|
||||||
public static Object getFrameworkConnectionConfig(ConnectorType type, Configuration configuration) {
|
public static Object getFrameworkConnectionConfig(Direction type, Configuration configuration) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION, JOB_CONFIG_FROM_FRAMEWORK_CONNECTION_KEY);
|
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION, JOB_CONFIG_FROM_FRAMEWORK_CONNECTION_KEY);
|
||||||
@ -235,7 +235,7 @@ public static Object getFrameworkJobConfig(Configuration configuration) {
|
|||||||
* @param job MapReduce Job object
|
* @param job MapReduce Job object
|
||||||
* @param schema Schema
|
* @param schema Schema
|
||||||
*/
|
*/
|
||||||
public static void setConnectorSchema(ConnectorType type, Job job, Schema schema) {
|
public static void setConnectorSchema(Direction type, Job job, Schema schema) {
|
||||||
if(schema != null) {
|
if(schema != null) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
@ -253,7 +253,7 @@ public static void setConnectorSchema(ConnectorType type, Job job, Schema schema
|
|||||||
* @param type The FROM or TO connector
|
* @param type The FROM or TO connector
|
||||||
* @param configuration MapReduce configuration object
|
* @param configuration MapReduce configuration object
|
||||||
*/
|
*/
|
||||||
public static Schema getConnectorSchema(ConnectorType type, Configuration configuration) {
|
public static Schema getConnectorSchema(Direction type, Configuration configuration) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
return getSchemaFromBytes(((JobConf) configuration).getCredentials().getSecretKey(SCHEMA_FROM_CONNECTOR_KEY));
|
return getSchemaFromBytes(((JobConf) configuration).getCredentials().getSecretKey(SCHEMA_FROM_CONNECTOR_KEY));
|
||||||
|
@ -19,12 +19,11 @@
|
|||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.job.JobConstants;
|
import org.apache.sqoop.job.JobConstants;
|
||||||
import org.apache.sqoop.job.PrefixContext;
|
import org.apache.sqoop.job.PrefixContext;
|
||||||
import org.apache.sqoop.job.etl.Destroyer;
|
import org.apache.sqoop.job.etl.Destroyer;
|
||||||
import org.apache.sqoop.job.etl.DestroyerContext;
|
import org.apache.sqoop.job.etl.DestroyerContext;
|
||||||
import org.apache.sqoop.model.MConnector;
|
|
||||||
import org.apache.sqoop.schema.Schema;
|
import org.apache.sqoop.schema.Schema;
|
||||||
import org.apache.sqoop.utils.ClassUtils;
|
import org.apache.sqoop.utils.ClassUtils;
|
||||||
|
|
||||||
@ -54,12 +53,12 @@ public static void executeDestroyer(boolean success, Configuration configuration
|
|||||||
|
|
||||||
// Objects that should be pass to the Destroyer execution
|
// Objects that should be pass to the Destroyer execution
|
||||||
PrefixContext subContext = new PrefixContext(configuration, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
|
PrefixContext subContext = new PrefixContext(configuration, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
|
||||||
Object fromConfigConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.FROM, configuration);
|
Object fromConfigConnection = ConfigurationUtils.getConnectorConnectionConfig(Direction.FROM, configuration);
|
||||||
Object fromConfigJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.FROM, configuration);
|
Object fromConfigJob = ConfigurationUtils.getConnectorJobConfig(Direction.FROM, configuration);
|
||||||
|
|
||||||
// Propagate connector schema in every case for now
|
// Propagate connector schema in every case for now
|
||||||
// TODO: Change to coditional choosing between Connector schemas.
|
// TODO: Change to coditional choosing between Connector schemas.
|
||||||
Schema schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, configuration);
|
Schema schema = ConfigurationUtils.getConnectorSchema(Direction.FROM, configuration);
|
||||||
|
|
||||||
DestroyerContext destroyerContext = new DestroyerContext(subContext, success, schema);
|
DestroyerContext destroyerContext = new DestroyerContext(subContext, success, schema);
|
||||||
|
|
||||||
|
@ -29,7 +29,7 @@
|
|||||||
import org.apache.hadoop.mapreduce.RecordReader;
|
import org.apache.hadoop.mapreduce.RecordReader;
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.job.JobConstants;
|
import org.apache.sqoop.job.JobConstants;
|
||||||
import org.apache.sqoop.job.MapreduceExecutionError;
|
import org.apache.sqoop.job.MapreduceExecutionError;
|
||||||
@ -37,7 +37,6 @@
|
|||||||
import org.apache.sqoop.job.etl.Partition;
|
import org.apache.sqoop.job.etl.Partition;
|
||||||
import org.apache.sqoop.job.etl.Partitioner;
|
import org.apache.sqoop.job.etl.Partitioner;
|
||||||
import org.apache.sqoop.job.etl.PartitionerContext;
|
import org.apache.sqoop.job.etl.PartitionerContext;
|
||||||
import org.apache.sqoop.model.MConnector;
|
|
||||||
import org.apache.sqoop.schema.Schema;
|
import org.apache.sqoop.schema.Schema;
|
||||||
import org.apache.sqoop.utils.ClassUtils;
|
import org.apache.sqoop.utils.ClassUtils;
|
||||||
|
|
||||||
@ -64,9 +63,9 @@ public List<InputSplit> getSplits(JobContext context)
|
|||||||
Partitioner partitioner = (Partitioner) ClassUtils.instantiate(partitionerName);
|
Partitioner partitioner = (Partitioner) ClassUtils.instantiate(partitionerName);
|
||||||
|
|
||||||
PrefixContext connectorContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
|
PrefixContext connectorContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
|
||||||
Object connectorConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.FROM, conf);
|
Object connectorConnection = ConfigurationUtils.getConnectorConnectionConfig(Direction.FROM, conf);
|
||||||
Object connectorJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.FROM, conf);
|
Object connectorJob = ConfigurationUtils.getConnectorJobConfig(Direction.FROM, conf);
|
||||||
Schema schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, conf);
|
Schema schema = ConfigurationUtils.getConnectorSchema(Direction.FROM, conf);
|
||||||
|
|
||||||
long maxPartitions = conf.getLong(JobConstants.JOB_ETL_EXTRACTOR_NUM, 10);
|
long maxPartitions = conf.getLong(JobConstants.JOB_ETL_EXTRACTOR_NUM, 10);
|
||||||
PartitionerContext partitionerContext = new PartitionerContext(connectorContext, maxPartitions, schema);
|
PartitionerContext partitionerContext = new PartitionerContext(connectorContext, maxPartitions, schema);
|
||||||
|
@ -26,7 +26,7 @@
|
|||||||
import org.apache.hadoop.io.NullWritable;
|
import org.apache.hadoop.io.NullWritable;
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
||||||
import org.apache.sqoop.job.JobConstants;
|
import org.apache.sqoop.job.JobConstants;
|
||||||
@ -35,7 +35,6 @@
|
|||||||
import org.apache.sqoop.job.etl.Extractor;
|
import org.apache.sqoop.job.etl.Extractor;
|
||||||
import org.apache.sqoop.job.etl.ExtractorContext;
|
import org.apache.sqoop.job.etl.ExtractorContext;
|
||||||
import org.apache.sqoop.etl.io.DataWriter;
|
import org.apache.sqoop.etl.io.DataWriter;
|
||||||
import org.apache.sqoop.model.MConnector;
|
|
||||||
import org.apache.sqoop.schema.Schema;
|
import org.apache.sqoop.schema.Schema;
|
||||||
import org.apache.sqoop.job.io.SqoopWritable;
|
import org.apache.sqoop.job.io.SqoopWritable;
|
||||||
import org.apache.sqoop.submission.counter.SqoopCounters;
|
import org.apache.sqoop.submission.counter.SqoopCounters;
|
||||||
@ -67,7 +66,7 @@ public void run(Context context) throws IOException, InterruptedException {
|
|||||||
|
|
||||||
// Propagate connector schema in every case for now
|
// Propagate connector schema in every case for now
|
||||||
// TODO: Change to coditional choosing between Connector schemas.
|
// TODO: Change to coditional choosing between Connector schemas.
|
||||||
Schema schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, conf);
|
Schema schema = ConfigurationUtils.getConnectorSchema(Direction.FROM, conf);
|
||||||
|
|
||||||
String intermediateDataFormatName = conf.get(JobConstants
|
String intermediateDataFormatName = conf.get(JobConstants
|
||||||
.INTERMEDIATE_DATA_FORMAT);
|
.INTERMEDIATE_DATA_FORMAT);
|
||||||
@ -82,8 +81,8 @@ public void run(Context context) throws IOException, InterruptedException {
|
|||||||
|
|
||||||
// Get configs for extractor
|
// Get configs for extractor
|
||||||
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
|
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
|
||||||
configConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.FROM, conf);
|
configConnection = ConfigurationUtils.getConnectorConnectionConfig(Direction.FROM, conf);
|
||||||
configJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.FROM, conf);
|
configJob = ConfigurationUtils.getConnectorJobConfig(Direction.FROM, conf);
|
||||||
|
|
||||||
SqoopSplit split = context.getCurrentKey();
|
SqoopSplit split = context.getCurrentKey();
|
||||||
ExtractorContext extractorContext = new ExtractorContext(subContext, new MapDataWriter(context), schema);
|
ExtractorContext extractorContext = new ExtractorContext(subContext, new MapDataWriter(context), schema);
|
||||||
|
@ -30,7 +30,7 @@
|
|||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
|
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
|
||||||
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
||||||
@ -40,7 +40,6 @@
|
|||||||
import org.apache.sqoop.job.etl.Loader;
|
import org.apache.sqoop.job.etl.Loader;
|
||||||
import org.apache.sqoop.job.etl.LoaderContext;
|
import org.apache.sqoop.job.etl.LoaderContext;
|
||||||
import org.apache.sqoop.etl.io.DataReader;
|
import org.apache.sqoop.etl.io.DataReader;
|
||||||
import org.apache.sqoop.model.MConnector;
|
|
||||||
import org.apache.sqoop.schema.Schema;
|
import org.apache.sqoop.schema.Schema;
|
||||||
import org.apache.sqoop.job.io.SqoopWritable;
|
import org.apache.sqoop.job.io.SqoopWritable;
|
||||||
import org.apache.sqoop.utils.ClassUtils;
|
import org.apache.sqoop.utils.ClassUtils;
|
||||||
@ -73,7 +72,7 @@ public SqoopOutputFormatLoadExecutor(JobContext jobctx) {
|
|||||||
producer = new SqoopRecordWriter();
|
producer = new SqoopRecordWriter();
|
||||||
data = (IntermediateDataFormat) ClassUtils.instantiate(context
|
data = (IntermediateDataFormat) ClassUtils.instantiate(context
|
||||||
.getConfiguration().get(JobConstants.INTERMEDIATE_DATA_FORMAT));
|
.getConfiguration().get(JobConstants.INTERMEDIATE_DATA_FORMAT));
|
||||||
data.setSchema(ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, context.getConfiguration()));
|
data.setSchema(ConfigurationUtils.getConnectorSchema(Direction.FROM, context.getConfiguration()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public RecordWriter<SqoopWritable, NullWritable> getRecordWriter() {
|
public RecordWriter<SqoopWritable, NullWritable> getRecordWriter() {
|
||||||
@ -229,11 +228,11 @@ public void run() {
|
|||||||
// Propagate connector schema in every case for now
|
// Propagate connector schema in every case for now
|
||||||
// TODO: Change to coditional choosing between Connector schemas.
|
// TODO: Change to coditional choosing between Connector schemas.
|
||||||
// @TODO(Abe): Maybe use TO schema?
|
// @TODO(Abe): Maybe use TO schema?
|
||||||
schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, conf);
|
schema = ConfigurationUtils.getConnectorSchema(Direction.FROM, conf);
|
||||||
|
|
||||||
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_TO_CONTEXT);
|
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_TO_CONTEXT);
|
||||||
configConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.TO, conf);
|
configConnection = ConfigurationUtils.getConnectorConnectionConfig(Direction.TO, conf);
|
||||||
configJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.TO, conf);
|
configJob = ConfigurationUtils.getConnectorJobConfig(Direction.TO, conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create loader context
|
// Create loader context
|
||||||
|
@ -38,8 +38,8 @@
|
|||||||
|
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.commons.lang.StringUtils;
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.ConnectorTypeError;
|
import org.apache.sqoop.common.DirectionError;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.model.MBooleanInput;
|
import org.apache.sqoop.model.MBooleanInput;
|
||||||
import org.apache.sqoop.model.MConnection;
|
import org.apache.sqoop.model.MConnection;
|
||||||
@ -150,9 +150,9 @@ private void insertFormsForConnector (MConnector mc, Connection conn) {
|
|||||||
MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
|
MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
|
||||||
|
|
||||||
// Register all jobs
|
// Register all jobs
|
||||||
registerForms(connectorId, ConnectorType.FROM, mc.getJobForms(ConnectorType.FROM).getForms(),
|
registerForms(connectorId, Direction.FROM, mc.getJobForms(Direction.FROM).getForms(),
|
||||||
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
|
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
|
||||||
registerForms(connectorId, ConnectorType.TO, mc.getJobForms(ConnectorType.TO).getForms(),
|
registerForms(connectorId, Direction.TO, mc.getJobForms(Direction.TO).getForms(),
|
||||||
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
|
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
|
||||||
|
|
||||||
} catch (SQLException ex) {
|
} catch (SQLException ex) {
|
||||||
@ -925,8 +925,8 @@ public void createJob(MJob job, Connection conn) {
|
|||||||
stmt = conn.prepareStatement(STMT_INSERT_JOB,
|
stmt = conn.prepareStatement(STMT_INSERT_JOB,
|
||||||
Statement.RETURN_GENERATED_KEYS);
|
Statement.RETURN_GENERATED_KEYS);
|
||||||
stmt.setString(1, job.getName());
|
stmt.setString(1, job.getName());
|
||||||
stmt.setLong(2, job.getConnectionId(ConnectorType.FROM));
|
stmt.setLong(2, job.getConnectionId(Direction.FROM));
|
||||||
stmt.setLong(3, job.getConnectionId(ConnectorType.TO));
|
stmt.setLong(3, job.getConnectionId(Direction.TO));
|
||||||
stmt.setBoolean(4, job.getEnabled());
|
stmt.setBoolean(4, job.getEnabled());
|
||||||
stmt.setString(5, job.getCreationUser());
|
stmt.setString(5, job.getCreationUser());
|
||||||
stmt.setTimestamp(6, new Timestamp(job.getCreationDate().getTime()));
|
stmt.setTimestamp(6, new Timestamp(job.getCreationDate().getTime()));
|
||||||
@ -949,7 +949,7 @@ public void createJob(MJob job, Connection conn) {
|
|||||||
|
|
||||||
createInputValues(STMT_INSERT_JOB_INPUT,
|
createInputValues(STMT_INSERT_JOB_INPUT,
|
||||||
jobId,
|
jobId,
|
||||||
job.getConnectorPart(ConnectorType.FROM).getForms(),
|
job.getConnectorPart(Direction.FROM).getForms(),
|
||||||
conn);
|
conn);
|
||||||
createInputValues(STMT_INSERT_JOB_INPUT,
|
createInputValues(STMT_INSERT_JOB_INPUT,
|
||||||
jobId,
|
jobId,
|
||||||
@ -957,7 +957,7 @@ public void createJob(MJob job, Connection conn) {
|
|||||||
conn);
|
conn);
|
||||||
createInputValues(STMT_INSERT_JOB_INPUT,
|
createInputValues(STMT_INSERT_JOB_INPUT,
|
||||||
jobId,
|
jobId,
|
||||||
job.getConnectorPart(ConnectorType.TO).getForms(),
|
job.getConnectorPart(Direction.TO).getForms(),
|
||||||
conn);
|
conn);
|
||||||
|
|
||||||
job.setPersistenceId(jobId);
|
job.setPersistenceId(jobId);
|
||||||
@ -995,7 +995,7 @@ public void updateJob(MJob job, Connection conn) {
|
|||||||
// And reinsert new values
|
// And reinsert new values
|
||||||
createInputValues(STMT_INSERT_JOB_INPUT,
|
createInputValues(STMT_INSERT_JOB_INPUT,
|
||||||
job.getPersistenceId(),
|
job.getPersistenceId(),
|
||||||
job.getConnectorPart(ConnectorType.FROM).getForms(),
|
job.getConnectorPart(Direction.FROM).getForms(),
|
||||||
conn);
|
conn);
|
||||||
createInputValues(STMT_INSERT_JOB_INPUT,
|
createInputValues(STMT_INSERT_JOB_INPUT,
|
||||||
job.getPersistenceId(),
|
job.getPersistenceId(),
|
||||||
@ -1791,7 +1791,7 @@ private List<MJob> loadJobs(PreparedStatement stmt,
|
|||||||
* @return short number of forms registered.
|
* @return short number of forms registered.
|
||||||
* @throws SQLException
|
* @throws SQLException
|
||||||
*/
|
*/
|
||||||
private short registerForms(Long connectorId, ConnectorType connectorType,
|
private short registerForms(Long connectorId, Direction direction,
|
||||||
List<MForm> forms, String type, PreparedStatement baseFormStmt,
|
List<MForm> forms, String type, PreparedStatement baseFormStmt,
|
||||||
PreparedStatement baseInputStmt)
|
PreparedStatement baseInputStmt)
|
||||||
throws SQLException {
|
throws SQLException {
|
||||||
@ -1803,10 +1803,10 @@ private short registerForms(Long connectorId, ConnectorType connectorType,
|
|||||||
} else {
|
} else {
|
||||||
baseFormStmt.setLong(1, connectorId);
|
baseFormStmt.setLong(1, connectorId);
|
||||||
}
|
}
|
||||||
if(connectorType == null) {
|
if(direction == null) {
|
||||||
baseFormStmt.setNull(2, Types.VARCHAR);
|
baseFormStmt.setNull(2, Types.VARCHAR);
|
||||||
} else {
|
} else {
|
||||||
baseFormStmt.setString(2, connectorType.name());
|
baseFormStmt.setString(2, direction.name());
|
||||||
}
|
}
|
||||||
baseFormStmt.setString(3, form.getName());
|
baseFormStmt.setString(3, form.getName());
|
||||||
baseFormStmt.setString(4, type);
|
baseFormStmt.setString(4, type);
|
||||||
@ -2155,7 +2155,7 @@ public void loadConnectorForms(List<MForm> connectionForms,
|
|||||||
connectionForms.add(mf);
|
connectionForms.add(mf);
|
||||||
break;
|
break;
|
||||||
case JOB:
|
case JOB:
|
||||||
ConnectorType type = ConnectorType.valueOf(operation);
|
Direction type = Direction.valueOf(operation);
|
||||||
List<MForm> jobForms;
|
List<MForm> jobForms;
|
||||||
switch(type) {
|
switch(type) {
|
||||||
case FROM:
|
case FROM:
|
||||||
@ -2167,7 +2167,7 @@ public void loadConnectorForms(List<MForm> connectionForms,
|
|||||||
break;
|
break;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
throw new SqoopException(ConnectorTypeError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
throw new SqoopException(DirectionError.CONNECTOR_TYPE_0000, "Connector type: " + type);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (jobForms.size() != formIndex) {
|
if (jobForms.size() != formIndex) {
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
|
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.audit.AuditLoggerManager;
|
import org.apache.sqoop.audit.AuditLoggerManager;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.connector.ConnectorManager;
|
import org.apache.sqoop.connector.ConnectorManager;
|
||||||
import org.apache.sqoop.connector.spi.SqoopConnector;
|
import org.apache.sqoop.connector.spi.SqoopConnector;
|
||||||
@ -165,26 +165,26 @@ private JsonBean createUpdateJob(RequestContext ctx, boolean update) {
|
|||||||
|
|
||||||
// Verify that user is not trying to spoof us
|
// Verify that user is not trying to spoof us
|
||||||
MJobForms fromConnectorForms = ConnectorManager.getInstance()
|
MJobForms fromConnectorForms = ConnectorManager.getInstance()
|
||||||
.getConnectorMetadata(job.getConnectorId(ConnectorType.FROM))
|
.getConnectorMetadata(job.getConnectorId(Direction.FROM))
|
||||||
.getJobForms(ConnectorType.FROM);
|
.getJobForms(Direction.FROM);
|
||||||
MJobForms toConnectorForms = ConnectorManager.getInstance()
|
MJobForms toConnectorForms = ConnectorManager.getInstance()
|
||||||
.getConnectorMetadata(job.getConnectorId(ConnectorType.TO))
|
.getConnectorMetadata(job.getConnectorId(Direction.TO))
|
||||||
.getJobForms(ConnectorType.TO);
|
.getJobForms(Direction.TO);
|
||||||
MJobForms frameworkForms = FrameworkManager.getInstance().getFramework()
|
MJobForms frameworkForms = FrameworkManager.getInstance().getFramework()
|
||||||
.getJobForms();
|
.getJobForms();
|
||||||
|
|
||||||
if(!fromConnectorForms.equals(job.getConnectorPart(ConnectorType.FROM))
|
if(!fromConnectorForms.equals(job.getConnectorPart(Direction.FROM))
|
||||||
|| !frameworkForms.equals(job.getFrameworkPart())
|
|| !frameworkForms.equals(job.getFrameworkPart())
|
||||||
|| !toConnectorForms.equals(job.getConnectorPart(ConnectorType.TO))) {
|
|| !toConnectorForms.equals(job.getConnectorPart(Direction.TO))) {
|
||||||
throw new SqoopException(ServerError.SERVER_0003,
|
throw new SqoopException(ServerError.SERVER_0003,
|
||||||
"Detected incorrect form structure");
|
"Detected incorrect form structure");
|
||||||
}
|
}
|
||||||
|
|
||||||
// Responsible connector for this session
|
// Responsible connector for this session
|
||||||
SqoopConnector fromConnector =
|
SqoopConnector fromConnector =
|
||||||
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.FROM));
|
ConnectorManager.getInstance().getConnector(job.getConnectorId(Direction.FROM));
|
||||||
SqoopConnector toConnector =
|
SqoopConnector toConnector =
|
||||||
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.TO));
|
ConnectorManager.getInstance().getConnector(job.getConnectorId(Direction.TO));
|
||||||
|
|
||||||
// Get validator objects
|
// Get validator objects
|
||||||
Validator fromConnectorValidator = fromConnector.getValidator();
|
Validator fromConnectorValidator = fromConnector.getValidator();
|
||||||
@ -193,15 +193,15 @@ private JsonBean createUpdateJob(RequestContext ctx, boolean update) {
|
|||||||
|
|
||||||
// We need translate forms to configuration objects
|
// We need translate forms to configuration objects
|
||||||
Object fromConnectorConfig = ClassUtils.instantiate(
|
Object fromConnectorConfig = ClassUtils.instantiate(
|
||||||
fromConnector.getJobConfigurationClass(ConnectorType.FROM));
|
fromConnector.getJobConfigurationClass(Direction.FROM));
|
||||||
Object frameworkConfig = ClassUtils.instantiate(
|
Object frameworkConfig = ClassUtils.instantiate(
|
||||||
FrameworkManager.getInstance().getJobConfigurationClass());
|
FrameworkManager.getInstance().getJobConfigurationClass());
|
||||||
Object toConnectorConfig = ClassUtils.instantiate(
|
Object toConnectorConfig = ClassUtils.instantiate(
|
||||||
toConnector.getJobConfigurationClass(ConnectorType.TO));
|
toConnector.getJobConfigurationClass(Direction.TO));
|
||||||
|
|
||||||
FormUtils.fromForms(job.getConnectorPart(ConnectorType.FROM).getForms(), fromConnectorConfig);
|
FormUtils.fromForms(job.getConnectorPart(Direction.FROM).getForms(), fromConnectorConfig);
|
||||||
FormUtils.fromForms(job.getFrameworkPart().getForms(), frameworkConfig);
|
FormUtils.fromForms(job.getFrameworkPart().getForms(), frameworkConfig);
|
||||||
FormUtils.fromForms(job.getConnectorPart(ConnectorType.TO).getForms(), toConnectorConfig);
|
FormUtils.fromForms(job.getConnectorPart(Direction.TO).getForms(), toConnectorConfig);
|
||||||
|
|
||||||
// Validate all parts
|
// Validate all parts
|
||||||
Validation fromConnectorValidation =
|
Validation fromConnectorValidation =
|
||||||
@ -262,7 +262,7 @@ private JsonBean getJobs(RequestContext ctx) {
|
|||||||
// Add associated resources into the bean
|
// Add associated resources into the bean
|
||||||
// @TODO(Abe): From/To.
|
// @TODO(Abe): From/To.
|
||||||
for( MJob job : jobs) {
|
for( MJob job : jobs) {
|
||||||
long connectorId = job.getConnectorId(ConnectorType.FROM);
|
long connectorId = job.getConnectorId(Direction.FROM);
|
||||||
if(!bean.hasConnectorBundle(connectorId)) {
|
if(!bean.hasConnectorBundle(connectorId)) {
|
||||||
bean.addConnectorBundle(connectorId,
|
bean.addConnectorBundle(connectorId,
|
||||||
ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
|
ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
|
||||||
@ -273,7 +273,7 @@ private JsonBean getJobs(RequestContext ctx) {
|
|||||||
|
|
||||||
MJob job = repository.findJob(jid);
|
MJob job = repository.findJob(jid);
|
||||||
// @TODO(Abe): From/To
|
// @TODO(Abe): From/To
|
||||||
long connectorId = job.getConnectorId(ConnectorType.FROM);
|
long connectorId = job.getConnectorId(Direction.FROM);
|
||||||
|
|
||||||
bean = new JobBean(job);
|
bean = new JobBean(job);
|
||||||
|
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
import jline.ConsoleReader;
|
import jline.ConsoleReader;
|
||||||
import org.apache.commons.cli.CommandLine;
|
import org.apache.commons.cli.CommandLine;
|
||||||
import org.apache.commons.cli.OptionBuilder;
|
import org.apache.commons.cli.OptionBuilder;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MJob;
|
import org.apache.sqoop.model.MJob;
|
||||||
import org.apache.sqoop.model.MPersistableEntity;
|
import org.apache.sqoop.model.MPersistableEntity;
|
||||||
import org.apache.sqoop.shell.core.Constants;
|
import org.apache.sqoop.shell.core.Constants;
|
||||||
@ -64,10 +64,10 @@ private Status cloneJob(Long jobId, List<String> args, boolean isInteractive) th
|
|||||||
job.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
|
job.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
|
||||||
|
|
||||||
ResourceBundle fromConnectorBundle = client.getResourceBundle(
|
ResourceBundle fromConnectorBundle = client.getResourceBundle(
|
||||||
job.getConnectorId(ConnectorType.FROM));
|
job.getConnectorId(Direction.FROM));
|
||||||
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
|
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
|
||||||
ResourceBundle toConnectorBundle = client.getResourceBundle(
|
ResourceBundle toConnectorBundle = client.getResourceBundle(
|
||||||
job.getConnectorId(ConnectorType.TO));
|
job.getConnectorId(Direction.TO));
|
||||||
|
|
||||||
Status status = Status.FINE;
|
Status status = Status.FINE;
|
||||||
|
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
import jline.ConsoleReader;
|
import jline.ConsoleReader;
|
||||||
import org.apache.commons.cli.CommandLine;
|
import org.apache.commons.cli.CommandLine;
|
||||||
import org.apache.commons.cli.OptionBuilder;
|
import org.apache.commons.cli.OptionBuilder;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MJob;
|
import org.apache.sqoop.model.MJob;
|
||||||
import org.apache.sqoop.shell.core.Constants;
|
import org.apache.sqoop.shell.core.Constants;
|
||||||
import org.apache.sqoop.shell.utils.FormDisplayer;
|
import org.apache.sqoop.shell.utils.FormDisplayer;
|
||||||
@ -75,9 +75,9 @@ private Status createJob(Long fromConnectionId, Long toConnectionId, List<String
|
|||||||
|
|
||||||
// @TODO(Abe): From/To.
|
// @TODO(Abe): From/To.
|
||||||
ResourceBundle fromConnectorBundle = client.getResourceBundle(
|
ResourceBundle fromConnectorBundle = client.getResourceBundle(
|
||||||
job.getConnectorId(ConnectorType.FROM));
|
job.getConnectorId(Direction.FROM));
|
||||||
ResourceBundle toConnectorBundle = client.getResourceBundle(
|
ResourceBundle toConnectorBundle = client.getResourceBundle(
|
||||||
job.getConnectorId(ConnectorType.TO));
|
job.getConnectorId(Direction.TO));
|
||||||
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
|
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
|
||||||
|
|
||||||
Status status = Status.FINE;
|
Status status = Status.FINE;
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
|
|
||||||
import org.apache.commons.cli.CommandLine;
|
import org.apache.commons.cli.CommandLine;
|
||||||
import org.apache.commons.cli.OptionBuilder;
|
import org.apache.commons.cli.OptionBuilder;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MJob;
|
import org.apache.sqoop.model.MJob;
|
||||||
import org.apache.sqoop.shell.core.Constants;
|
import org.apache.sqoop.shell.core.Constants;
|
||||||
import org.apache.sqoop.shell.utils.TableDisplayer;
|
import org.apache.sqoop.shell.utils.TableDisplayer;
|
||||||
@ -82,9 +82,9 @@ private void showSummary() {
|
|||||||
ids.add(String.valueOf(job.getPersistenceId()));
|
ids.add(String.valueOf(job.getPersistenceId()));
|
||||||
names.add(job.getName());
|
names.add(job.getName());
|
||||||
fromConnectors.add(String.valueOf(
|
fromConnectors.add(String.valueOf(
|
||||||
job.getConnectorId(ConnectorType.FROM)));
|
job.getConnectorId(Direction.FROM)));
|
||||||
toConnectors.add(String.valueOf(
|
toConnectors.add(String.valueOf(
|
||||||
job.getConnectorId(ConnectorType.TO)));
|
job.getConnectorId(Direction.TO)));
|
||||||
availabilities.add(String.valueOf(job.getEnabled()));
|
availabilities.add(String.valueOf(job.getEnabled()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -121,15 +121,15 @@ private void displayJob(MJob job) {
|
|||||||
formatter.format(job.getLastUpdateDate())
|
formatter.format(job.getLastUpdateDate())
|
||||||
);
|
);
|
||||||
printlnResource(Constants.RES_SHOW_PROMPT_JOB_XID_CID_INFO,
|
printlnResource(Constants.RES_SHOW_PROMPT_JOB_XID_CID_INFO,
|
||||||
job.getConnectionId(ConnectorType.FROM),
|
job.getConnectionId(Direction.FROM),
|
||||||
job.getConnectorId(ConnectorType.FROM));
|
job.getConnectorId(Direction.FROM));
|
||||||
|
|
||||||
// Display connector part
|
// Display connector part
|
||||||
displayForms(job.getConnectorPart(ConnectorType.FROM).getForms(),
|
displayForms(job.getConnectorPart(Direction.FROM).getForms(),
|
||||||
client.getResourceBundle(job.getConnectorId(ConnectorType.FROM)));
|
client.getResourceBundle(job.getConnectorId(Direction.FROM)));
|
||||||
displayForms(job.getFrameworkPart().getForms(),
|
displayForms(job.getFrameworkPart().getForms(),
|
||||||
client.getFrameworkResourceBundle());
|
client.getFrameworkResourceBundle());
|
||||||
displayForms(job.getConnectorPart(ConnectorType.TO).getForms(),
|
displayForms(job.getConnectorPart(Direction.TO).getForms(),
|
||||||
client.getResourceBundle(job.getConnectorId(ConnectorType.TO)));
|
client.getResourceBundle(job.getConnectorId(Direction.TO)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
import jline.ConsoleReader;
|
import jline.ConsoleReader;
|
||||||
import org.apache.commons.cli.CommandLine;
|
import org.apache.commons.cli.CommandLine;
|
||||||
import org.apache.commons.cli.OptionBuilder;
|
import org.apache.commons.cli.OptionBuilder;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MJob;
|
import org.apache.sqoop.model.MJob;
|
||||||
import org.apache.sqoop.shell.core.Constants;
|
import org.apache.sqoop.shell.core.Constants;
|
||||||
import org.apache.sqoop.shell.utils.FormDisplayer;
|
import org.apache.sqoop.shell.utils.FormDisplayer;
|
||||||
@ -64,10 +64,10 @@ private Status updateJob(Long jobId, List<String> args, boolean isInteractive) t
|
|||||||
MJob job = client.getJob(jobId);
|
MJob job = client.getJob(jobId);
|
||||||
|
|
||||||
ResourceBundle fromConnectorBundle = client.getResourceBundle(
|
ResourceBundle fromConnectorBundle = client.getResourceBundle(
|
||||||
job.getConnectorId(ConnectorType.FROM));
|
job.getConnectorId(Direction.FROM));
|
||||||
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
|
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
|
||||||
ResourceBundle toConnectorBundle = client.getResourceBundle(
|
ResourceBundle toConnectorBundle = client.getResourceBundle(
|
||||||
job.getConnectorId(ConnectorType.TO));
|
job.getConnectorId(Direction.TO));
|
||||||
|
|
||||||
Status status = Status.FINE;
|
Status status = Status.FINE;
|
||||||
|
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
package org.apache.sqoop.shell.utils;
|
package org.apache.sqoop.shell.utils;
|
||||||
|
|
||||||
import org.apache.commons.lang.StringUtils;
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MAccountableEntity;
|
import org.apache.sqoop.model.MAccountableEntity;
|
||||||
import org.apache.sqoop.model.MBooleanInput;
|
import org.apache.sqoop.model.MBooleanInput;
|
||||||
import org.apache.sqoop.model.MConnection;
|
import org.apache.sqoop.model.MConnection;
|
||||||
@ -70,13 +70,13 @@ public static void displayFormMetadataDetails(MConnector connector,
|
|||||||
|
|
||||||
// @TODO(Abe): Validate From/To output is correct.
|
// @TODO(Abe): Validate From/To output is correct.
|
||||||
displayFormsMetadata(
|
displayFormsMetadata(
|
||||||
connector.getJobForms(ConnectorType.FROM).getForms(),
|
connector.getJobForms(Direction.FROM).getForms(),
|
||||||
ConnectorType.FROM.toString() + " " + resourceString(Constants.RES_FORMDISPLAYER_JOB),
|
Direction.FROM.toString() + " " + resourceString(Constants.RES_FORMDISPLAYER_JOB),
|
||||||
bundle);
|
bundle);
|
||||||
|
|
||||||
displayFormsMetadata(
|
displayFormsMetadata(
|
||||||
connector.getJobForms(ConnectorType.TO).getForms(),
|
connector.getJobForms(Direction.TO).getForms(),
|
||||||
ConnectorType.TO.toString() + " " + resourceString(Constants.RES_FORMDISPLAYER_JOB),
|
Direction.TO.toString() + " " + resourceString(Constants.RES_FORMDISPLAYER_JOB),
|
||||||
bundle);
|
bundle);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -153,9 +153,9 @@ public static void displayFormWarning(MAccountableEntity entity) {
|
|||||||
formList.addAll(connection.getFrameworkPart().getForms());
|
formList.addAll(connection.getFrameworkPart().getForms());
|
||||||
} else if(entity instanceof MJob) {
|
} else if(entity instanceof MJob) {
|
||||||
MJob job = (MJob) entity;
|
MJob job = (MJob) entity;
|
||||||
formList.addAll(job.getConnectorPart(ConnectorType.FROM).getForms());
|
formList.addAll(job.getConnectorPart(Direction.FROM).getForms());
|
||||||
formList.addAll(job.getFrameworkPart().getForms());
|
formList.addAll(job.getFrameworkPart().getForms());
|
||||||
formList.addAll(job.getConnectorPart(ConnectorType.TO).getForms());
|
formList.addAll(job.getConnectorPart(Direction.TO).getForms());
|
||||||
}
|
}
|
||||||
for(MForm form : formList) {
|
for(MForm form : formList) {
|
||||||
if(form.getValidationStatus() == Status.ACCEPTABLE) {
|
if(form.getValidationStatus() == Status.ACCEPTABLE) {
|
||||||
|
@ -21,7 +21,7 @@
|
|||||||
|
|
||||||
import org.apache.commons.cli.CommandLine;
|
import org.apache.commons.cli.CommandLine;
|
||||||
import org.apache.commons.lang.StringUtils;
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MBooleanInput;
|
import org.apache.sqoop.model.MBooleanInput;
|
||||||
import org.apache.sqoop.model.MConnection;
|
import org.apache.sqoop.model.MConnection;
|
||||||
import org.apache.sqoop.model.MEnumInput;
|
import org.apache.sqoop.model.MEnumInput;
|
||||||
@ -69,7 +69,7 @@ public static boolean fillJob(CommandLine line,
|
|||||||
|
|
||||||
// Fill in data from user
|
// Fill in data from user
|
||||||
return fillForms(line,
|
return fillForms(line,
|
||||||
job.getConnectorPart(ConnectorType.FROM).getForms(),
|
job.getConnectorPart(Direction.FROM).getForms(),
|
||||||
job.getFrameworkPart().getForms());
|
job.getFrameworkPart().getForms());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -94,11 +94,11 @@ public static boolean fillJob(ConsoleReader reader,
|
|||||||
|
|
||||||
// Fill in data from user
|
// Fill in data from user
|
||||||
return fillForms(reader,
|
return fillForms(reader,
|
||||||
job.getConnectorPart(ConnectorType.FROM).getForms(),
|
job.getConnectorPart(Direction.FROM).getForms(),
|
||||||
fromConnectorBundle,
|
fromConnectorBundle,
|
||||||
job.getFrameworkPart().getForms(),
|
job.getFrameworkPart().getForms(),
|
||||||
frameworkBundle,
|
frameworkBundle,
|
||||||
job.getConnectorPart(ConnectorType.TO).getForms(),
|
job.getConnectorPart(Direction.TO).getForms(),
|
||||||
toConnectorBundle);
|
toConnectorBundle);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -909,7 +909,7 @@ public static void printConnectionValidationMessages(MConnection connection) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static void printJobValidationMessages(MJob job) {
|
public static void printJobValidationMessages(MJob job) {
|
||||||
for (MForm form : job.getConnectorPart(ConnectorType.FROM).getForms()) {
|
for (MForm form : job.getConnectorPart(Direction.FROM).getForms()) {
|
||||||
for (MInput<?> input : form.getInputs()) {
|
for (MInput<?> input : form.getInputs()) {
|
||||||
printValidationMessage(input, true);
|
printValidationMessage(input, true);
|
||||||
}
|
}
|
||||||
@ -919,7 +919,7 @@ public static void printJobValidationMessages(MJob job) {
|
|||||||
printValidationMessage(input, true);
|
printValidationMessage(input, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for (MForm form : job.getConnectorPart(ConnectorType.TO).getForms()) {
|
for (MForm form : job.getConnectorPart(Direction.TO).getForms()) {
|
||||||
for (MInput<?> input : form.getInputs()) {
|
for (MInput<?> input : form.getInputs()) {
|
||||||
printValidationMessage(input, true);
|
printValidationMessage(input, true);
|
||||||
}
|
}
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
|
|
||||||
import org.apache.commons.cli.Option;
|
import org.apache.commons.cli.Option;
|
||||||
import org.apache.commons.cli.OptionBuilder;
|
import org.apache.commons.cli.OptionBuilder;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.model.MJob;
|
import org.apache.sqoop.model.MJob;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -35,13 +35,13 @@ public void prepareOptions(MJob job) {
|
|||||||
.withLongOpt("name")
|
.withLongOpt("name")
|
||||||
.hasArg()
|
.hasArg()
|
||||||
.create());
|
.create());
|
||||||
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart(ConnectorType.FROM).getForms())) {
|
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart(Direction.FROM).getForms())) {
|
||||||
this.addOption(option);
|
this.addOption(option);
|
||||||
}
|
}
|
||||||
for (Option option : FormOptions.getFormsOptions("framework", job.getFrameworkPart().getForms())) {
|
for (Option option : FormOptions.getFormsOptions("framework", job.getFrameworkPart().getForms())) {
|
||||||
this.addOption(option);
|
this.addOption(option);
|
||||||
}
|
}
|
||||||
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart(ConnectorType.TO).getForms())) {
|
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart(Direction.TO).getForms())) {
|
||||||
this.addOption(option);
|
this.addOption(option);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.ResourceBundle;
|
import java.util.ResourceBundle;
|
||||||
|
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
|
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
|
||||||
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
|
||||||
import org.apache.sqoop.job.etl.From;
|
import org.apache.sqoop.job.etl.From;
|
||||||
@ -53,7 +53,7 @@ public abstract class SqoopConnector {
|
|||||||
/**
|
/**
|
||||||
* @return Get job configuration class for given type or null if not supported
|
* @return Get job configuration class for given type or null if not supported
|
||||||
*/
|
*/
|
||||||
public abstract Class getJobConfigurationClass(ConnectorType jobType);
|
public abstract Class getJobConfigurationClass(Direction jobType);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return an <tt>From</tt> that provides classes for performing import.
|
* @return an <tt>From</tt> that provides classes for performing import.
|
||||||
|
@ -26,7 +26,7 @@
|
|||||||
import org.apache.hadoop.mapred.RunningJob;
|
import org.apache.hadoop.mapred.RunningJob;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.sqoop.common.ConnectorType;
|
import org.apache.sqoop.common.Direction;
|
||||||
import org.apache.sqoop.common.MapContext;
|
import org.apache.sqoop.common.MapContext;
|
||||||
import org.apache.sqoop.common.SqoopException;
|
import org.apache.sqoop.common.SqoopException;
|
||||||
import org.apache.sqoop.execution.mapreduce.MRSubmissionRequest;
|
import org.apache.sqoop.execution.mapreduce.MRSubmissionRequest;
|
||||||
@ -164,7 +164,7 @@ public boolean submit(SubmissionRequest generalRequest) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Serialize connector context as a sub namespace
|
// Serialize connector context as a sub namespace
|
||||||
for(Map.Entry<String, String> entry : request.getConnectorContext(ConnectorType.FROM)) {
|
for(Map.Entry<String, String> entry : request.getConnectorContext(Direction.FROM)) {
|
||||||
if (entry.getValue() == null) {
|
if (entry.getValue() == null) {
|
||||||
LOG.warn("Ignoring null connector context value for key " + entry.getKey());
|
LOG.warn("Ignoring null connector context value for key " + entry.getKey());
|
||||||
continue;
|
continue;
|
||||||
@ -174,7 +174,7 @@ public boolean submit(SubmissionRequest generalRequest) {
|
|||||||
entry.getValue());
|
entry.getValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
for(Map.Entry<String, String> entry : request.getConnectorContext(ConnectorType.TO)) {
|
for(Map.Entry<String, String> entry : request.getConnectorContext(Direction.TO)) {
|
||||||
if (entry.getValue() == null) {
|
if (entry.getValue() == null) {
|
||||||
LOG.warn("Ignoring null connector context value for key " + entry.getKey());
|
LOG.warn("Ignoring null connector context value for key " + entry.getKey());
|
||||||
continue;
|
continue;
|
||||||
@ -200,15 +200,15 @@ public boolean submit(SubmissionRequest generalRequest) {
|
|||||||
Job job = new Job(configuration);
|
Job job = new Job(configuration);
|
||||||
|
|
||||||
// And finally put all configuration objects to credentials cache
|
// And finally put all configuration objects to credentials cache
|
||||||
ConfigurationUtils.setConnectorConnectionConfig(ConnectorType.FROM, job, request.getConnectorConnectionConfig(ConnectorType.FROM));
|
ConfigurationUtils.setConnectorConnectionConfig(Direction.FROM, job, request.getConnectorConnectionConfig(Direction.FROM));
|
||||||
ConfigurationUtils.setConnectorJobConfig(ConnectorType.FROM, job, request.getConnectorJobConfig(ConnectorType.FROM));
|
ConfigurationUtils.setConnectorJobConfig(Direction.FROM, job, request.getConnectorJobConfig(Direction.FROM));
|
||||||
ConfigurationUtils.setConnectorConnectionConfig(ConnectorType.TO, job, request.getConnectorConnectionConfig(ConnectorType.TO));
|
ConfigurationUtils.setConnectorConnectionConfig(Direction.TO, job, request.getConnectorConnectionConfig(Direction.TO));
|
||||||
ConfigurationUtils.setConnectorJobConfig(ConnectorType.TO, job, request.getConnectorJobConfig(ConnectorType.TO));
|
ConfigurationUtils.setConnectorJobConfig(Direction.TO, job, request.getConnectorJobConfig(Direction.TO));
|
||||||
ConfigurationUtils.setFrameworkConnectionConfig(ConnectorType.FROM, job, request.getFrameworkConnectionConfig(ConnectorType.FROM));
|
ConfigurationUtils.setFrameworkConnectionConfig(Direction.FROM, job, request.getFrameworkConnectionConfig(Direction.FROM));
|
||||||
ConfigurationUtils.setFrameworkConnectionConfig(ConnectorType.TO, job, request.getFrameworkConnectionConfig(ConnectorType.TO));
|
ConfigurationUtils.setFrameworkConnectionConfig(Direction.TO, job, request.getFrameworkConnectionConfig(Direction.TO));
|
||||||
ConfigurationUtils.setFrameworkJobConfig(job, request.getConfigFrameworkJob());
|
ConfigurationUtils.setFrameworkJobConfig(job, request.getConfigFrameworkJob());
|
||||||
// @TODO(Abe): Persist TO schema.
|
// @TODO(Abe): Persist TO schema.
|
||||||
ConfigurationUtils.setConnectorSchema(ConnectorType.FROM, job, request.getSummary().getConnectorSchema());
|
ConfigurationUtils.setConnectorSchema(Direction.FROM, job, request.getSummary().getConnectorSchema());
|
||||||
|
|
||||||
if(request.getJobName() != null) {
|
if(request.getJobName() != null) {
|
||||||
job.setJobName("Sqoop: " + request.getJobName());
|
job.setJobName("Sqoop: " + request.getJobName());
|
||||||
|
Loading…
Reference in New Issue
Block a user