SQOOP-1376: Sqoop2: From/To: Refactor connector interface

(Abraham Elmahrek via Jarek Jarcec Cecho)
This commit is contained in:
Jarek Jarcec Cecho 2014-08-08 13:11:35 -07:00 коммит произвёл Abraham Elmahrek
Родитель f18a29bf17
Коммит 0ad7401bb5
73 изменённых файлов: 1693 добавлений и 1249 удалений

Просмотреть файл

@ -18,10 +18,12 @@
package org.apache.sqoop.client;
import org.apache.sqoop.client.request.SqoopRequests;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.json.ConnectionValidationBean;
import org.apache.sqoop.json.ConnectorBean;
import org.apache.sqoop.json.FrameworkBean;
import org.apache.sqoop.json.ValidationBean;
import org.apache.sqoop.json.JobValidationBean;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MConnection;
import org.apache.sqoop.model.MConnector;
@ -351,21 +353,24 @@ public class SqoopClient {
}
/**
* Create new job of given type and for given connection.
* Create new job the for given connections.
*
* @param xid Connection id
* @param type Job type
* @param fromXid From Connection id
* @param toXid To Connection id
* @return
*/
public MJob newJob(long xid, MJob.Type type) {
MConnection connection = getConnection(xid);
public MJob newJob(long fromXid, long toXid) {
MConnection fromConnection = getConnection(fromXid);
MConnection toConnection = getConnection(toXid);
return new MJob(
connection.getConnectorId(),
connection.getPersistenceId(),
type,
getConnector(connection.getConnectorId()).getJobForms(type),
getFramework().getJobForms(type)
fromConnection.getConnectorId(),
toConnection.getConnectorId(),
fromConnection.getPersistenceId(),
toConnection.getPersistenceId(),
getConnector(fromConnection.getConnectorId()).getJobForms(ConnectorType.FROM),
getConnector(fromConnection.getConnectorId()).getJobForms(ConnectorType.TO),
getFramework().getJobForms()
);
}
@ -529,7 +534,7 @@ public class SqoopClient {
return requests.readHistory(jid).getSubmissions();
}
private Status applyValidations(ValidationBean bean, MConnection connection) {
private Status applyValidations(ConnectionValidationBean bean, MConnection connection) {
Validation connector = bean.getConnectorValidation();
Validation framework = bean.getFrameworkValidation();
@ -544,18 +549,25 @@ public class SqoopClient {
return Status.getWorstStatus(connector.getStatus(), framework.getStatus());
}
private Status applyValidations(ValidationBean bean, MJob job) {
Validation connector = bean.getConnectorValidation();
private Status applyValidations(JobValidationBean bean, MJob job) {
Validation fromConnector = bean.getConnectorValidation(ConnectorType.FROM);
Validation toConnector = bean.getConnectorValidation(ConnectorType.TO);
Validation framework = bean.getFrameworkValidation();
FormUtils.applyValidation(job.getConnectorPart().getForms(), connector);
// @TODO(Abe): From/To validation.
FormUtils.applyValidation(
job.getConnectorPart(ConnectorType.FROM).getForms(),
fromConnector);
FormUtils.applyValidation(job.getFrameworkPart().getForms(), framework);
FormUtils.applyValidation(
job.getConnectorPart(ConnectorType.TO).getForms(),
toConnector);
Long id = bean.getId();
if(id != null) {
job.setPersistenceId(id);
}
return Status.getWorstStatus(connector.getStatus(), framework.getStatus());
return Status.getWorstStatus(fromConnector.getStatus(), framework.getStatus(), toConnector.getStatus());
}
}

Просмотреть файл

@ -18,7 +18,7 @@
package org.apache.sqoop.client.request;
import org.apache.sqoop.json.ConnectionBean;
import org.apache.sqoop.json.ValidationBean;
import org.apache.sqoop.json.ConnectionValidationBean;
import org.apache.sqoop.model.MConnection;
import org.json.simple.JSONObject;
import org.json.simple.JSONValue;
@ -49,7 +49,7 @@ public class ConnectionRequest extends Request {
return connectionBean;
}
public ValidationBean create(String serverUrl, MConnection connection) {
public ConnectionValidationBean create(String serverUrl, MConnection connection) {
ConnectionBean connectionBean = new ConnectionBean(connection);
@ -59,13 +59,13 @@ public class ConnectionRequest extends Request {
String response = super.post(serverUrl + RESOURCE,
connectionJson.toJSONString());
ValidationBean validationBean = new ValidationBean();
validationBean.restore((JSONObject) JSONValue.parse(response));
ConnectionValidationBean connectionValidationBean = new ConnectionValidationBean();
connectionValidationBean.restore((JSONObject) JSONValue.parse(response));
return validationBean;
return connectionValidationBean;
}
public ValidationBean update(String serverUrl, MConnection connection) {
public ConnectionValidationBean update(String serverUrl, MConnection connection) {
ConnectionBean connectionBean = new ConnectionBean(connection);
@ -76,10 +76,10 @@ public class ConnectionRequest extends Request {
+ connection.getPersistenceId(),
connectionJson.toJSONString());
ValidationBean validationBean = new ValidationBean();
validationBean.restore((JSONObject) JSONValue.parse(response));
ConnectionValidationBean connectionValidationBean = new ConnectionValidationBean();
connectionValidationBean.restore((JSONObject) JSONValue.parse(response));
return validationBean;
return connectionValidationBean;
}
public void delete(String serverUrl, Long id) {

Просмотреть файл

@ -18,7 +18,7 @@
package org.apache.sqoop.client.request;
import org.apache.sqoop.json.JobBean;
import org.apache.sqoop.json.ValidationBean;
import org.apache.sqoop.json.JobValidationBean;
import org.apache.sqoop.model.MJob;
import org.json.simple.JSONObject;
import org.json.simple.JSONValue;
@ -49,7 +49,7 @@ public class JobRequest extends Request {
return jobBean;
}
public ValidationBean create(String serverUrl, MJob job) {
public JobValidationBean create(String serverUrl, MJob job) {
JobBean jobBean = new JobBean(job);
@ -59,13 +59,13 @@ public class JobRequest extends Request {
String response = super.post(serverUrl + RESOURCE,
jobJson.toJSONString());
ValidationBean validationBean = new ValidationBean();
JobValidationBean validationBean = new JobValidationBean();
validationBean.restore((JSONObject) JSONValue.parse(response));
return validationBean;
}
public ValidationBean update(String serverUrl, MJob job) {
public JobValidationBean update(String serverUrl, MJob job) {
JobBean jobBean = new JobBean(job);
@ -75,7 +75,7 @@ public class JobRequest extends Request {
String response = super.put(serverUrl + RESOURCE + job.getPersistenceId(),
jobJson.toJSONString());
ValidationBean validationBean = new ValidationBean();
JobValidationBean validationBean = new JobValidationBean();
validationBean.restore((JSONObject) JSONValue.parse(response));
return validationBean;

Просмотреть файл

@ -18,16 +18,14 @@
package org.apache.sqoop.client.request;
import org.apache.sqoop.json.ConnectionBean;
import org.apache.sqoop.json.ConnectionValidationBean;
import org.apache.sqoop.json.ConnectorBean;
import org.apache.sqoop.json.FrameworkBean;
import org.apache.sqoop.json.JobBean;
import org.apache.sqoop.json.JobValidationBean;
import org.apache.sqoop.json.SubmissionBean;
import org.apache.sqoop.json.ValidationBean;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MConnection;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.validation.Status;
import org.apache.sqoop.validation.Validation;
/**
* Unified class for all request objects.
@ -94,7 +92,7 @@ public class SqoopRequests {
return getConnectorRequest().read(serverUrl, cid);
}
public ValidationBean createConnection(MConnection connection) {
public ConnectionValidationBean createConnection(MConnection connection) {
return getConnectionRequest().create(serverUrl, connection);
}
@ -102,7 +100,7 @@ public class SqoopRequests {
return getConnectionRequest().read(serverUrl, connectionId);
}
public ValidationBean updateConnection(MConnection connection) {
public ConnectionValidationBean updateConnection(MConnection connection) {
return getConnectionRequest().update(serverUrl, connection);
}
@ -114,7 +112,7 @@ public class SqoopRequests {
getConnectionRequest().delete(serverUrl, xid);
}
public ValidationBean createJob(MJob job) {
public JobValidationBean createJob(MJob job) {
return getJobRequest().create(serverUrl, job);
}
@ -122,7 +120,7 @@ public class SqoopRequests {
return getJobRequest().read(serverUrl, jobId);
}
public ValidationBean updateJob(MJob job) {
public JobValidationBean updateJob(MJob job) {
return getJobRequest().update(serverUrl, job);
}

Просмотреть файл

@ -0,0 +1,30 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.sqoop.common;
/**
* Connectors will have configurations for FROM and TO.
* If the connector is being used to extract data FROM,
* then the connector type will be FROM. If the connector
* is being used to load data TO, then the connector type
* will be TO.
*/
public enum ConnectorType {
FROM,
TO
}

Просмотреть файл

@ -30,7 +30,7 @@ import java.util.Map;
* part of validated entity. Optionally validation bean can also transfer
* created persistent id in case that new entity was created.
*/
public class ValidationBean implements JsonBean {
public class ConnectionValidationBean implements JsonBean {
private static final String ID = "id";
private static final String FRAMEWORK = "framework";
@ -44,7 +44,7 @@ public class ValidationBean implements JsonBean {
private Validation frameworkValidation;
// For "extract"
public ValidationBean(Validation connector, Validation framework) {
public ConnectionValidationBean(Validation connector, Validation framework) {
this();
this.connectorValidation = connector;
@ -52,7 +52,7 @@ public class ValidationBean implements JsonBean {
}
// For "restore"
public ValidationBean() {
public ConnectionValidationBean() {
id = null;
}

Просмотреть файл

@ -24,8 +24,8 @@ import java.util.Map;
import java.util.ResourceBundle;
import java.util.Set;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MConnectionForms;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MJobForms;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.model.MForm;
@ -73,14 +73,13 @@ public class ConnectorBean implements JsonBean {
object.put(NAME, connector.getUniqueName());
object.put(CLASS, connector.getClassName());
object.put(VERSION, connector.getVersion());
object.put(CON_FORMS, extractForms(connector.getConnectionForms().getForms(), skipSensitive));
JSONObject jobForms = new JSONObject();
for (MJobForms job : connector.getAllJobsForms().values()) {
jobForms.put(job.getType().name(), extractForms(job.getForms(), skipSensitive));
}
object.put(JOB_FORMS, jobForms);
object.put(JOB_FORMS, new JSONObject());
((JSONObject)object.get(JOB_FORMS)).put(
ConnectorType.FROM, extractForms(connector.getJobForms(ConnectorType.FROM).getForms(), skipSensitive));
((JSONObject)object.get(JOB_FORMS)).put(
ConnectorType.TO, extractForms(connector.getJobForms(ConnectorType.TO).getForms(), skipSensitive));
array.add(object);
}
@ -119,17 +118,17 @@ public class ConnectorBean implements JsonBean {
List<MForm> connForms = restoreForms((JSONArray) object.get(CON_FORMS));
JSONObject jobJson = (JSONObject) object.get(JOB_FORMS);
List<MJobForms> jobs = new ArrayList<MJobForms>();
for( Map.Entry entry : (Set<Map.Entry>) jobJson.entrySet()) {
MJob.Type type = MJob.Type.valueOf((String) entry.getKey());
JSONArray fromJobJson = (JSONArray)jobJson.get(ConnectorType.FROM.name());
JSONArray toJobJson = (JSONArray)jobJson.get(ConnectorType.TO.name());
List<MForm> fromJobForms =
restoreForms(fromJobJson);
List<MForm> toJobForms =
restoreForms(toJobJson);
MJobForms fromJob = new MJobForms(fromJobForms);
MJobForms toJob = new MJobForms(toJobForms);
MConnectionForms connection = new MConnectionForms(connForms);
List<MForm> jobForms =
restoreForms((JSONArray) jobJson.get(entry.getKey()));
jobs.add(new MJobForms(type, jobForms));
}
MConnector connector = new MConnector(uniqueName, className, version, new MConnectionForms(connForms), jobs);
MConnector connector = new MConnector(uniqueName, className, version, connection, fromJob, toJob);
connector.setPersistenceId(connectorId);
connectors.add(connector);

Просмотреть файл

@ -18,6 +18,7 @@
package org.apache.sqoop.json;
import org.apache.sqoop.model.MConnectionForms;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.model.MForm;
import org.apache.sqoop.model.MFramework;
import org.apache.sqoop.model.MJob;
@ -65,13 +66,10 @@ public class FrameworkBean implements JsonBean {
@SuppressWarnings("unchecked")
@Override
public JSONObject extract(boolean skipSensitive) {
// @TODO(Abe): Add From/To connection forms.
JSONArray conForms =
extractForms(framework.getConnectionForms().getForms(), skipSensitive);
JSONObject jobForms = new JSONObject();
for (MJobForms job : framework.getAllJobsForms().values()) {
jobForms.put(job.getType().name(), extractForms(job.getForms(), skipSensitive));
}
JSONArray jobForms = extractForms(framework.getJobForms().getForms(), skipSensitive);
JSONObject result = new JSONObject();
result.put(ID, framework.getPersistenceId());
@ -89,22 +87,13 @@ public class FrameworkBean implements JsonBean {
String frameworkVersion = (String) jsonObject.get(FRAMEWORK_VERSION);
List<MForm> connForms = restoreForms((JSONArray) jsonObject.get(CON_FORMS));
List<MForm> jobForms = restoreForms((JSONArray) jsonObject.get(JOB_FORMS));
JSONObject jobForms = (JSONObject) jsonObject.get(JOB_FORMS);
List<MJobForms> jobs = new ArrayList<MJobForms>();
for( Map.Entry entry : (Set<Map.Entry>) jobForms.entrySet()) {
//TODO(jarcec): Handle situation when server is supporting operation
// that client do not know (server do have newer version than client)
MJob.Type type = MJob.Type.valueOf((String) entry.getKey());
List<MForm> job = restoreForms((JSONArray) entry.getValue());
jobs.add(new MJobForms(type, job));
}
framework = new MFramework(new MConnectionForms(connForms), jobs,
frameworkVersion);
// @TODO(Abe): Get From/To connection forms.
framework = new MFramework(
new MConnectionForms(connForms),
new MJobForms(jobForms),
frameworkVersion);
framework.setPersistenceId(id);
bundle = restoreResourceBundle((JSONObject) jsonObject.get(RESOURCES));

Просмотреть файл

@ -17,6 +17,7 @@
*/
package org.apache.sqoop.json;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MForm;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MJobForms;
@ -42,10 +43,12 @@ public class JobBean implements JsonBean {
private static final String ALL = "all";
private static final String ID = "id";
private static final String NAME = "name";
private static final String TYPE = "type";
private static final String CONNECTION_ID = "connection-id";
private static final String CONNECTOR_ID = "connector-id";
private static final String CONNECTOR_PART = "connector";
private static final String FROM_CONNECTION_ID = "from-connection-id";
private static final String TO_CONNECTION_ID = "to-connection-id";
private static final String FROM_CONNECTOR_ID = "from-connector-id";
private static final String TO_CONNECTOR_ID = "to-connector-id";
private static final String FROM_CONNECTOR_PART = "from-connector";
private static final String TO_CONNECTOR_PART = "to-connector";
private static final String FRAMEWORK_PART = "framework";
// Compulsory
@ -106,16 +109,19 @@ public class JobBean implements JsonBean {
object.put(ID, job.getPersistenceId());
object.put(NAME, job.getName());
object.put(TYPE, job.getType().name());
object.put(ENABLED, job.getEnabled());
object.put(CREATION_USER, job.getCreationUser());
object.put(CREATION_DATE, job.getCreationDate().getTime());
object.put(UPDATE_USER, job.getLastUpdateUser());
object.put(UPDATE_DATE, job.getLastUpdateDate().getTime());
object.put(CONNECTION_ID, job.getConnectionId());
object.put(CONNECTOR_ID, job.getConnectorId());
object.put(CONNECTOR_PART,
extractForms(job.getConnectorPart().getForms(), skipSensitive));
object.put(FROM_CONNECTION_ID, job.getConnectionId(ConnectorType.FROM));
object.put(TO_CONNECTION_ID, job.getConnectionId(ConnectorType.TO));
object.put(FROM_CONNECTOR_ID, job.getConnectorId(ConnectorType.FROM));
object.put(TO_CONNECTOR_ID, job.getConnectorId(ConnectorType.TO));
object.put(FROM_CONNECTOR_PART,
extractForms(job.getConnectorPart(ConnectorType.FROM).getForms(),skipSensitive));
object.put(TO_CONNECTOR_PART,
extractForms(job.getConnectorPart(ConnectorType.TO).getForms(), skipSensitive));
object.put(FRAMEWORK_PART,
extractForms(job.getFrameworkPart().getForms(), skipSensitive));
@ -151,23 +157,26 @@ public class JobBean implements JsonBean {
for (Object obj : array) {
JSONObject object = (JSONObject) obj;
long connectorId = (Long) object.get(CONNECTOR_ID);
long connectionId = (Long) object.get(CONNECTION_ID);
JSONArray connectorPart = (JSONArray) object.get(CONNECTOR_PART);
long fromConnectorId = (Long) object.get(FROM_CONNECTOR_ID);
long toConnectorId = (Long) object.get(TO_CONNECTOR_ID);
long fromConnectionId = (Long) object.get(FROM_CONNECTION_ID);
long toConnectionId = (Long) object.get(TO_CONNECTION_ID);
JSONArray fromConnectorPart = (JSONArray) object.get(FROM_CONNECTOR_PART);
JSONArray toConnectorPart = (JSONArray) object.get(TO_CONNECTOR_PART);
JSONArray frameworkPart = (JSONArray) object.get(FRAMEWORK_PART);
String stringType = (String) object.get(TYPE);
MJob.Type type = MJob.Type.valueOf(stringType);
List<MForm> connectorForms = restoreForms(connectorPart);
List<MForm> fromConnectorParts = restoreForms(fromConnectorPart);
List<MForm> toConnectorParts = restoreForms(toConnectorPart);
List<MForm> frameworkForms = restoreForms(frameworkPart);
MJob job = new MJob(
connectorId,
connectionId,
type,
new MJobForms(type, connectorForms),
new MJobForms(type, frameworkForms)
fromConnectorId,
toConnectorId,
fromConnectionId,
toConnectionId,
new MJobForms(fromConnectorParts),
new MJobForms(toConnectorParts),
new MJobForms(frameworkForms)
);
job.setPersistenceId((Long) object.get(ID));

Просмотреть файл

@ -0,0 +1,157 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.sqoop.json;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.validation.Status;
import org.apache.sqoop.validation.Validation;
import org.json.simple.JSONObject;
import java.util.HashMap;
import java.util.Map;
/**
* Bean for sending validations across network. This bean will move two
* validation objects at one time - one for connector and second for framework
* part of validated entity. Optionally validation bean can also transfer
* created persistent id in case that new entity was created.
*/
public class JobValidationBean implements JsonBean {
private static final String ID = "id";
private static final String FRAMEWORK = "framework";
private static final String CONNECTOR = "connector";
private static final String FROM = "from";
private static final String TO = "to";
private static final String STATUS = "status";
private static final String MESSAGE = "message";
private static final String MESSAGES = "messages";
private Long id;
private Map<ConnectorType, Validation> connectorValidation;
private Validation frameworkValidation;
// For "extract"
public JobValidationBean(Validation fromConnector, Validation framework, Validation toConnector) {
this();
this.connectorValidation = new HashMap<ConnectorType, Validation>();
this.connectorValidation.put(ConnectorType.FROM, fromConnector);
this.connectorValidation.put(ConnectorType.TO, toConnector);
this.frameworkValidation = framework;
}
// For "restore"
public JobValidationBean() {
id = null;
connectorValidation = new HashMap<ConnectorType, Validation>();
}
public Validation getConnectorValidation(ConnectorType type) {
return connectorValidation.get(type);
}
public Validation getFrameworkValidation() {
return frameworkValidation;
}
public void setId(Long id) {
this.id = id;
}
public Long getId() {
return id;
}
@SuppressWarnings("unchecked")
public JSONObject extract(boolean skipSensitive) {
JSONObject object = new JSONObject();
JSONObject connectorObject = new JSONObject();
// Optionally transfer id
if(id != null) {
object.put(ID, id);
}
connectorObject.put(FROM, extractValidation(getConnectorValidation(ConnectorType.FROM)));
connectorObject.put(TO, extractValidation(getConnectorValidation(ConnectorType.TO)));
object.put(FRAMEWORK, extractValidation(frameworkValidation));
object.put(CONNECTOR, connectorObject);
return object;
}
@SuppressWarnings("unchecked")
private JSONObject extractValidation(Validation validation) {
JSONObject object = new JSONObject();
object.put(STATUS, validation.getStatus().name());
JSONObject jsonMessages = new JSONObject();
Map<Validation.FormInput, Validation.Message> messages = validation.getMessages();
for(Map.Entry<Validation.FormInput, Validation.Message> entry : messages.entrySet()) {
JSONObject jsonEntry = new JSONObject();
jsonEntry.put(STATUS, entry.getValue().getStatus().name());
jsonEntry.put(MESSAGE, entry.getValue().getMessage());
jsonMessages.put(entry.getKey(), jsonEntry);
}
object.put(MESSAGES, jsonMessages);
return object;
}
@Override
public void restore(JSONObject jsonObject) {
// Optional and accepting NULLs
id = (Long) jsonObject.get(ID);
JSONObject jsonConnectorObject = (JSONObject)jsonObject.get(CONNECTOR);
connectorValidation.put(ConnectorType.FROM, restoreValidation(
(JSONObject)jsonConnectorObject.get(FROM)));
connectorValidation.put(ConnectorType.TO, restoreValidation(
(JSONObject)jsonConnectorObject.get(TO)));
frameworkValidation = restoreValidation(
(JSONObject)jsonObject.get(FRAMEWORK));
}
public Validation restoreValidation(JSONObject jsonObject) {
JSONObject jsonMessages = (JSONObject) jsonObject.get(MESSAGES);
Map<Validation.FormInput, Validation.Message> messages
= new HashMap<Validation.FormInput, Validation.Message>();
for(Object key : jsonMessages.keySet()) {
JSONObject jsonMessage = (JSONObject) jsonMessages.get(key);
Status status = Status.valueOf((String) jsonMessage.get(STATUS));
String stringMessage = (String) jsonMessage.get(MESSAGE);
Validation.Message message
= new Validation.Message(status, stringMessage);
messages.put(new Validation.FormInput((String)key), message);
}
Status status = Status.valueOf((String) jsonObject.get(STATUS));
return new Validation(status, messages);
}
}

Просмотреть файл

@ -17,8 +17,10 @@
*/
package org.apache.sqoop.model;
import java.util.ArrayList;
import java.util.List;
import java.util.HashMap;
import java.util.Map;
import org.apache.sqoop.common.ConnectorType;
/**
* Connector metadata.
@ -26,14 +28,23 @@ import java.util.List;
* Includes unique id that identifies connector in metadata store, unique human
* readable name, corresponding name and all forms for all supported job types.
*/
public final class MConnector extends MFramework {
public final class MConnector extends MPersistableEntity implements MClonable {
private final String uniqueName;
private final String className;
private final MConnectionForms connectionForms;
private final Map<ConnectorType, MJobForms> jobForms;
String version;
public MConnector(String uniqueName, String className, String version,
MConnectionForms connectionForms, List<MJobForms> jobForms) {
super(connectionForms, jobForms, version);
public MConnector(String uniqueName, String className,
String version, MConnectionForms connectionForms,
MJobForms fromJobForms, MJobForms toJobForms) {
this.jobForms = new HashMap<ConnectorType, MJobForms>();
this.version = version;
this.connectionForms = connectionForms;
this.jobForms.put(ConnectorType.FROM, fromJobForms);
this.jobForms.put(ConnectorType.TO, toJobForms);
if (uniqueName == null || className == null) {
throw new NullPointerException();
@ -57,10 +68,8 @@ public final class MConnector extends MFramework {
sb.append(uniqueName).append(":").append(getPersistenceId()).append(":");
sb.append(className);
sb.append(", ").append(getConnectionForms().toString());
for(MJobForms entry: getAllJobsForms().values()) {
sb.append(entry.toString());
}
sb.append(", ").append(getJobForms(ConnectorType.FROM).toString());
sb.append(", ").append(getJobForms(ConnectorType.TO).toString());
return sb.toString();
}
@ -78,32 +87,49 @@ public final class MConnector extends MFramework {
return uniqueName.equals(mc.uniqueName)
&& className.equals(mc.className)
&& version.equals(mc.version)
&& super.equals(other);
&& connectionForms.equals(mc.getConnectionForms())
&& jobForms.get(ConnectorType.FROM).equals(mc.getJobForms(ConnectorType.FROM))
&& jobForms.get(ConnectorType.TO).equals(mc.getJobForms(ConnectorType.TO));
}
@Override
public int hashCode() {
int result = super.hashCode();
int result = getConnectionForms().hashCode();
result = 31 * result + getJobForms(ConnectorType.FROM).hashCode();
result = 31 * result + getJobForms(ConnectorType.TO).hashCode();
result = 31 * result + version.hashCode();
result = 31 * result + uniqueName.hashCode();
result = 31 * result + className.hashCode();
return result;
}
@Override
public MConnector clone(boolean cloneWithValue) {
//Connector never have any values filled
cloneWithValue = false;
List<MJobForms> copyJobForms = null;
if(this.getAllJobsForms()!=null) {
copyJobForms = new ArrayList<MJobForms>();
for(MJobForms entry: this.getAllJobsForms().values()) {
copyJobForms.add(entry.clone(cloneWithValue));
}
}
MConnector copy = new MConnector(this.getUniqueName(), this.getClassName(), this.getVersion(),
this.getConnectionForms().clone(cloneWithValue), copyJobForms);
MConnector copy = new MConnector(
this.getUniqueName(),
this.getClassName(),
this.getVersion(),
this.getConnectionForms().clone(cloneWithValue),
this.getJobForms(ConnectorType.FROM).clone(cloneWithValue),
this.getJobForms(ConnectorType.TO).clone(cloneWithValue));
copy.setPersistenceId(this.getPersistenceId());
return copy;
}
public MConnectionForms getConnectionForms() {
return connectionForms;
}
public MJobForms getJobForms(ConnectorType type) {
return jobForms.get(type);
}
public String getVersion() {
return version;
}
public void setVersion(String version) {
this.version = version;
}
}

Просмотреть файл

@ -17,38 +17,21 @@
*/
package org.apache.sqoop.model;
import org.apache.sqoop.common.SqoopException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Metadata describing framework options for connection and job for each
* supported job type.
* Metadata describing framework options for connection and jobForms for each
* supported jobForms type.
*/
public class MFramework extends MPersistableEntity implements MClonable {
private final MConnectionForms connectionForms;
private final Map<MJob.Type, MJobForms> jobs;
private final MJobForms jobForms;
String version;
public MFramework(MConnectionForms connectionForms, List<MJobForms> jobForms,
public MFramework(MConnectionForms connectionForms, MJobForms jobForms,
String version) {
this.version = version;
this.connectionForms = connectionForms;
this.jobs = new HashMap<MJob.Type, MJobForms>();
for (MJobForms job : jobForms) {
MJob.Type type = job.getType();
if(this.jobs.containsKey(type)) {
throw new SqoopException(ModelError.MODEL_001, "Duplicate entry for"
+ " jobForms type " + job.getType().name());
}
this.jobs.put(type, job);
}
this.jobForms = jobForms;
}
@Override
@ -57,9 +40,7 @@ public class MFramework extends MPersistableEntity implements MClonable {
sb.append(getPersistenceId()).append(":");
sb.append("version = " + version);
sb.append(", ").append(connectionForms.toString());
for(MJobForms entry: jobs.values()) {
sb.append(entry.toString());
}
sb.append(jobForms.toString());
return sb.toString();
}
@ -77,16 +58,13 @@ public class MFramework extends MPersistableEntity implements MClonable {
MFramework mo = (MFramework) other;
return version.equals(mo.getVersion()) &&
connectionForms.equals(mo.connectionForms) &&
jobs.equals(mo.jobs);
jobForms.equals(mo.jobForms);
}
@Override
public int hashCode() {
int result = connectionForms.hashCode();
for(MJobForms entry: jobs.values()) {
result = 31 * result + entry.hashCode();
}
result = 31 * result + jobForms.hashCode();
result = 31 * result + version.hashCode();
return result;
}
@ -95,27 +73,16 @@ public class MFramework extends MPersistableEntity implements MClonable {
return connectionForms;
}
public Map<MJob.Type, MJobForms> getAllJobsForms() {
return jobs;
}
public MJobForms getJobForms(MJob.Type type) {
return jobs.get(type);
public MJobForms getJobForms() {
return jobForms;
}
@Override
public MFramework clone(boolean cloneWithValue) {
//Framework never have any values filled
cloneWithValue = false;
List<MJobForms> copyJobForms = null;
if(this.getAllJobsForms()!=null) {
copyJobForms = new ArrayList<MJobForms>();
for(MJobForms entry: this.getAllJobsForms().values()) {
copyJobForms.add(entry.clone(cloneWithValue));
}
}
MFramework copy = new MFramework(this.getConnectionForms().clone(cloneWithValue),
copyJobForms, this.version);
this.getJobForms().clone(cloneWithValue), this.version);
copy.setPersistenceId(this.getPersistenceId());
return copy;
}

Просмотреть файл

@ -17,19 +17,16 @@
*/
package org.apache.sqoop.model;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.common.ConnectorType;
import java.util.HashMap;
import java.util.Map;
/**
* Model describing entire job object including both connector and
* framework part.
*/
public class MJob extends MAccountableEntity implements MClonable {
public static enum Type {
IMPORT,
EXPORT,
}
/**
* Connector reference.
*
@ -37,46 +34,47 @@ public class MJob extends MAccountableEntity implements MClonable {
* dependency through connection object, but having this dependency explicitly
* carried along helps a lot.
*/
private final long connectorId;
private final Map<ConnectorType, Long> connectorIds;
/**
* Corresponding connection object.
* Corresponding connection objects for connector.
*/
private final long connectionId;
private final Map<ConnectorType, Long> connectionIds;
/**
* User name for this object
*/
private String name;
/**
* Job type
*/
private final Type type;
private final MJobForms connectorPart;
private final Map<ConnectorType, MJobForms> connectorParts;
private final MJobForms frameworkPart;
/**
* Default constructor to build new MJob model.
*
* @param connectorId Connector id
* @param connectionId Connection id
* @param type Job type
* @param connectorPart Connector forms
* @param fromConnectorId Connector id
* @param fromConnectionId Connection id
* @param fromPart From Connector forms
* @param toPart To Connector forms
* @param frameworkPart Framework forms
*/
public MJob(long connectorId,
long connectionId,
Type type,
MJobForms connectorPart,
public MJob(long fromConnectorId,
long toConnectorId,
long fromConnectionId,
long toConnectionId,
MJobForms fromPart,
MJobForms toPart,
MJobForms frameworkPart) {
this.connectorId = connectorId;
this.connectionId = connectionId;
this.type = type;
this.connectorPart = connectorPart;
connectorIds = new HashMap<ConnectorType, Long>();
connectorIds.put(ConnectorType.FROM, fromConnectorId);
connectorIds.put(ConnectorType.TO, toConnectorId);
connectionIds = new HashMap<ConnectorType, Long>();
connectionIds.put(ConnectorType.FROM, fromConnectionId);
connectionIds.put(ConnectorType.TO, toConnectionId);
connectorParts = new HashMap<ConnectorType, MJobForms>();
connectorParts.put(ConnectorType.FROM, fromPart);
connectorParts.put(ConnectorType.TO, toPart);
this.frameworkPart = frameworkPart;
verifyFormsOfSameType();
}
/**
@ -85,7 +83,10 @@ public class MJob extends MAccountableEntity implements MClonable {
* @param other MConnection model to copy
*/
public MJob(MJob other) {
this(other, other.connectorPart.clone(true), other.frameworkPart.clone(true));
this(other,
other.getConnectorPart(ConnectorType.FROM).clone(true),
other.getConnectorPart(ConnectorType.TO).clone(true),
other.frameworkPart.clone(true));
}
/**
@ -95,34 +96,31 @@ public class MJob extends MAccountableEntity implements MClonable {
* used otherwise.
*
* @param other MJob model to copy
* @param connectorPart Connector forms
* @param fromPart From Connector forms
* @param frameworkPart Framework forms
* @param toPart To Connector forms
*/
public MJob(MJob other, MJobForms connectorPart, MJobForms frameworkPart) {
public MJob(MJob other, MJobForms fromPart, MJobForms frameworkPart, MJobForms toPart) {
super(other);
this.connectionId = other.connectionId;
this.connectorId = other.connectorId;
this.type = other.type;
connectorIds = new HashMap<ConnectorType, Long>();
connectorIds.put(ConnectorType.FROM, other.getConnectorId(ConnectorType.FROM));
connectorIds.put(ConnectorType.TO, other.getConnectorId(ConnectorType.TO));
connectionIds = new HashMap<ConnectorType, Long>();
connectorIds.put(ConnectorType.FROM, other.getConnectionId(ConnectorType.FROM));
connectorIds.put(ConnectorType.TO, other.getConnectionId(ConnectorType.TO));
connectorParts = new HashMap<ConnectorType, MJobForms>();
connectorParts.put(ConnectorType.FROM, fromPart);
connectorParts.put(ConnectorType.TO, toPart);
this.name = other.name;
this.connectorPart = connectorPart;
this.frameworkPart = frameworkPart;
verifyFormsOfSameType();
}
private void verifyFormsOfSameType() {
if (type != connectorPart.getType() || type != frameworkPart.getType()) {
throw new SqoopException(ModelError.MODEL_002,
"Incompatible types, job: " + type.name()
+ ", connector part: " + connectorPart.getType().name()
+ ", framework part: " + frameworkPart.getType().name()
);
}
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("job connector-part: ");
sb.append(connectorPart).append(", framework-part: ").append(frameworkPart);
StringBuilder sb = new StringBuilder("job");
sb.append(" connector-from-part: ").append(getConnectorPart(ConnectorType.FROM));
sb.append(", connector-to-part: ").append(getConnectorPart(ConnectorType.TO));
sb.append(", framework-part: ").append(frameworkPart);
return sb.toString();
}
@ -135,32 +133,35 @@ public class MJob extends MAccountableEntity implements MClonable {
this.name = name;
}
public long getConnectionId() {
return connectionId;
public long getConnectionId(ConnectorType type) {
return connectionIds.get(type);
}
public long getConnectorId() {
return connectorId;
public long getConnectorId(ConnectorType type) {
return connectorIds.get(type);
}
public MJobForms getConnectorPart() {
return connectorPart;
public MJobForms getConnectorPart(ConnectorType type) {
return connectorParts.get(type);
}
public MJobForms getFrameworkPart() {
return frameworkPart;
}
public Type getType() {
return type;
}
@Override
public MJob clone(boolean cloneWithValue) {
if(cloneWithValue) {
return new MJob(this);
} else {
return new MJob(connectorId, connectionId, type, connectorPart.clone(false), frameworkPart.clone(false));
return new MJob(
getConnectorId(ConnectorType.FROM),
getConnectorId(ConnectorType.TO),
getConnectionId(ConnectorType.FROM),
getConnectionId(ConnectorType.TO),
getConnectorPart(ConnectorType.FROM).clone(false),
getConnectorPart(ConnectorType.TO).clone(false),
frameworkPart.clone(false));
}
}
@ -175,11 +176,13 @@ public class MJob extends MAccountableEntity implements MClonable {
}
MJob job = (MJob)object;
return (job.connectorId == this.connectorId)
&& (job.connectionId == this.connectionId)
return (job.getConnectorId(ConnectorType.FROM) == this.getConnectorId(ConnectorType.FROM))
&& (job.getConnectorId(ConnectorType.TO) == this.getConnectorId(ConnectorType.TO))
&& (job.getConnectionId(ConnectorType.FROM) == this.getConnectionId(ConnectorType.FROM))
&& (job.getConnectionId(ConnectorType.TO) == this.getConnectionId(ConnectorType.TO))
&& (job.getPersistenceId() == this.getPersistenceId())
&& (job.type.equals(this.type))
&& (job.connectorPart.equals(this.connectorPart))
&& (job.getConnectorPart(ConnectorType.FROM).equals(this.getConnectorPart(ConnectorType.FROM)))
&& (job.getConnectorPart(ConnectorType.TO).equals(this.getConnectorPart(ConnectorType.TO)))
&& (job.frameworkPart.equals(this.frameworkPart));
}
}

Просмотреть файл

@ -20,28 +20,12 @@ package org.apache.sqoop.model;
import java.util.List;
/**
* Metadata describing all required information to build up an job
* object for one part. Both connector and framework need to supply this object
* to build up entire job.
* Metadata describing all required information to build a job
* object with two connectors and a framework.
*/
public class MJobForms extends MFormList {
private final MJob.Type type;
public MJobForms(MJob.Type type, List<MForm> forms) {
public MJobForms(List<MForm> forms) {
super(forms);
this.type = type;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("Job type: ").append(type.name());
sb.append(super.toString());
return sb.toString();
}
public MJob.Type getType() {
return type;
}
@Override
@ -55,19 +39,17 @@ public class MJobForms extends MFormList {
}
MJobForms mj = (MJobForms) other;
return type.equals(mj.type) && super.equals(mj);
return super.equals(mj);
}
@Override
public int hashCode() {
int result = super.hashCode();
result = 31 * result + type.hashCode();
return result;
return super.hashCode();
}
@Override
public MJobForms clone(boolean cloneWithValue) {
MJobForms copy = new MJobForms(this.type, super.clone(cloneWithValue).getForms());
MJobForms copy = new MJobForms(super.clone(cloneWithValue).getForms());
return copy;
}
}

Просмотреть файл

@ -20,33 +20,33 @@ package org.apache.sqoop.connector.jdbc;
import java.util.Locale;
import java.util.ResourceBundle;
import org.apache.sqoop.common.ConnectorType;
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.ExportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
import org.apache.sqoop.connector.spi.MetadataUpgrader;
import org.apache.sqoop.job.etl.Exporter;
import org.apache.sqoop.job.etl.Importer;
import org.apache.sqoop.job.etl.From;
import org.apache.sqoop.job.etl.To;
import org.apache.sqoop.connector.spi.SqoopConnector;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.validation.Validator;
public class GenericJdbcConnector extends SqoopConnector {
private static GenericJdbcValidator genericJdbcValidator = new GenericJdbcValidator();
private static final Importer IMPORTER = new Importer(
GenericJdbcImportInitializer.class,
GenericJdbcImportPartitioner.class,
GenericJdbcImportExtractor.class,
GenericJdbcImportDestroyer.class);
private static final From FROM = new From(
GenericJdbcFromInitializer.class,
GenericJdbcPartitioner.class,
GenericJdbcExtractor.class,
GenericJdbcFromDestroyer.class);
private static final Exporter EXPORTER = new Exporter(
GenericJdbcExportInitializer.class,
GenericJdbcExportLoader.class,
GenericJdbcExportDestroyer.class);
private static final To TO = new To(
GenericJdbcToInitializer.class,
GenericJdbcLoader.class,
GenericJdbcToDestroyer.class);
/**
@ -72,25 +72,25 @@ public class GenericJdbcConnector extends SqoopConnector {
}
@Override
public Class getJobConfigurationClass(MJob.Type jobType) {
public Class getJobConfigurationClass(ConnectorType jobType) {
switch (jobType) {
case IMPORT:
return ImportJobConfiguration.class;
case EXPORT:
return ExportJobConfiguration.class;
case FROM:
return FromJobConfiguration.class;
case TO:
return ToJobConfiguration.class;
default:
return null;
}
}
@Override
public Importer getImporter() {
return IMPORTER;
public From getFrom() {
return FROM;
}
@Override
public Exporter getExporter() {
return EXPORTER;
public To getTo() {
return TO;
}
@Override

Просмотреть файл

@ -42,8 +42,10 @@ public final class GenericJdbcConnectorConstants {
public static final String CONNECTOR_JDBC_PARTITION_MAXVALUE =
PREFIX_CONNECTOR_JDBC_CONFIG + "partition.maxvalue";
public static final String CONNECTOR_JDBC_DATA_SQL =
PREFIX_CONNECTOR_JDBC_CONFIG + "data.sql";
public static final String CONNECTOR_FROM_JDBC_DATA_SQL =
PREFIX_CONNECTOR_JDBC_CONFIG + "from.data.sql";
public static final String CONNECTOR_TO_JDBC_DATA_SQL =
PREFIX_CONNECTOR_JDBC_CONFIG + "to.data.sql";
public static final String SQL_CONDITIONS_TOKEN = "${CONDITIONS}";

Просмотреть файл

@ -24,24 +24,24 @@ import java.sql.SQLException;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
import org.apache.sqoop.job.etl.ExtractorContext;
import org.apache.sqoop.job.etl.Extractor;
public class GenericJdbcImportExtractor extends Extractor<ConnectionConfiguration, ImportJobConfiguration, GenericJdbcImportPartition> {
public class GenericJdbcExtractor extends Extractor<ConnectionConfiguration, FromJobConfiguration, GenericJdbcPartition> {
public static final Logger LOG = Logger.getLogger(GenericJdbcImportExtractor.class);
public static final Logger LOG = Logger.getLogger(GenericJdbcExtractor.class);
private long rowsRead = 0;
@Override
public void extract(ExtractorContext context, ConnectionConfiguration connection, ImportJobConfiguration job, GenericJdbcImportPartition partition) {
public void extract(ExtractorContext context, ConnectionConfiguration connection, FromJobConfiguration job, GenericJdbcPartition partition) {
String driver = connection.connection.jdbcDriver;
String url = connection.connection.connectionString;
String username = connection.connection.username;
String password = connection.connection.password;
GenericJdbcExecutor executor = new GenericJdbcExecutor(driver, url, username, password);
String query = context.getString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_DATA_SQL);
String query = context.getString(GenericJdbcConnectorConstants.CONNECTOR_FROM_JDBC_DATA_SQL);
String conditions = partition.getConditions();
query = query.replace(GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN, conditions);
LOG.info("Using query: " + query);

Просмотреть файл

@ -19,17 +19,17 @@ package org.apache.sqoop.connector.jdbc;
import org.apache.log4j.Logger;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
import org.apache.sqoop.job.etl.Destroyer;
import org.apache.sqoop.job.etl.DestroyerContext;
public class GenericJdbcImportDestroyer extends Destroyer<ConnectionConfiguration, ImportJobConfiguration> {
public class GenericJdbcFromDestroyer extends Destroyer<ConnectionConfiguration, FromJobConfiguration> {
private static final Logger LOG =
Logger.getLogger(GenericJdbcImportDestroyer.class);
Logger.getLogger(GenericJdbcFromDestroyer.class);
@Override
public void destroy(DestroyerContext context, ConnectionConfiguration connection, ImportJobConfiguration job) {
public void destroy(DestroyerContext context, ConnectionConfiguration connection, FromJobConfiguration job) {
LOG.info("Running generic JDBC connector destroyer");
}

Просмотреть файл

@ -28,7 +28,7 @@ import org.apache.log4j.Logger;
import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
import org.apache.sqoop.connector.jdbc.util.SqlTypesUtils;
import org.apache.sqoop.job.Constants;
import org.apache.sqoop.job.etl.Initializer;
@ -37,15 +37,15 @@ import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.schema.type.Column;
import org.apache.sqoop.utils.ClassUtils;
public class GenericJdbcImportInitializer extends Initializer<ConnectionConfiguration, ImportJobConfiguration> {
public class GenericJdbcFromInitializer extends Initializer<ConnectionConfiguration, FromJobConfiguration> {
private static final Logger LOG =
Logger.getLogger(GenericJdbcImportInitializer.class);
Logger.getLogger(GenericJdbcFromInitializer.class);
private GenericJdbcExecutor executor;
@Override
public void initialize(InitializerContext context, ConnectionConfiguration connection, ImportJobConfiguration job) {
public void initialize(InitializerContext context, ConnectionConfiguration connection, FromJobConfiguration job) {
configureJdbcProperties(context.getContext(), connection, job);
try {
configurePartitionProperties(context.getContext(), connection, job);
@ -56,7 +56,7 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
}
@Override
public List<String> getJars(InitializerContext context, ConnectionConfiguration connection, ImportJobConfiguration job) {
public List<String> getJars(InitializerContext context, ConnectionConfiguration connection, FromJobConfiguration job) {
List<String> jars = new LinkedList<String>();
jars.add(ClassUtils.jarForClass(connection.connection.jdbcDriver));
@ -65,14 +65,14 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
}
@Override
public Schema getSchema(InitializerContext context, ConnectionConfiguration connectionConfiguration, ImportJobConfiguration importJobConfiguration) {
configureJdbcProperties(context.getContext(), connectionConfiguration, importJobConfiguration);
public Schema getSchema(InitializerContext context, ConnectionConfiguration connectionConfiguration, FromJobConfiguration fromJobConfiguration) {
configureJdbcProperties(context.getContext(), connectionConfiguration, fromJobConfiguration);
String schemaName = importJobConfiguration.table.tableName;
String schemaName = fromJobConfiguration.table.tableName;
if(schemaName == null) {
schemaName = "Query";
} else if(importJobConfiguration.table.schemaName != null) {
schemaName = importJobConfiguration.table.schemaName + "." + schemaName;
} else if(fromJobConfiguration.table.schemaName != null) {
schemaName = fromJobConfiguration.table.schemaName + "." + schemaName;
}
Schema schema = new Schema(schemaName);
@ -80,8 +80,8 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
ResultSetMetaData rsmt = null;
try {
rs = executor.executeQuery(
context.getString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_DATA_SQL)
.replace(GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN, "1 = 0")
context.getString(GenericJdbcConnectorConstants.CONNECTOR_FROM_JDBC_DATA_SQL)
.replace(GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN, "1 = 0")
);
rsmt = rs.getMetaData();
@ -114,7 +114,7 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
}
}
private void configureJdbcProperties(MutableContext context, ConnectionConfiguration connectionConfig, ImportJobConfiguration jobConfig) {
private void configureJdbcProperties(MutableContext context, ConnectionConfiguration connectionConfig, FromJobConfiguration jobConfig) {
String driver = connectionConfig.connection.jdbcDriver;
String url = connectionConfig.connection.connectionString;
String username = connectionConfig.connection.username;
@ -126,7 +126,7 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
executor = new GenericJdbcExecutor(driver, url, username, password);
}
private void configurePartitionProperties(MutableContext context, ConnectionConfiguration connectionConfig, ImportJobConfiguration jobConfig) {
private void configurePartitionProperties(MutableContext context, ConnectionConfiguration connectionConfig, FromJobConfiguration jobConfig) {
// ----- configure column name -----
String partitionColumnName = jobConfig.table.partitionColumn;
@ -231,7 +231,7 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
}
}
private void configureTableProperties(MutableContext context, ConnectionConfiguration connectionConfig, ImportJobConfiguration jobConfig) {
private void configureTableProperties(MutableContext context, ConnectionConfiguration connectionConfig, FromJobConfiguration jobConfig) {
String dataSql;
String fieldNames;
@ -316,7 +316,7 @@ public class GenericJdbcImportInitializer extends Initializer<ConnectionConfigur
LOG.info("Using dataSql: " + dataSql);
LOG.info("Field names: " + fieldNames);
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_DATA_SQL, dataSql);
context.setString(GenericJdbcConnectorConstants.CONNECTOR_FROM_JDBC_DATA_SQL, dataSql);
context.setString(Constants.JOB_ETL_FIELD_NAMES, fieldNames);
}
}

Просмотреть файл

@ -18,11 +18,11 @@
package org.apache.sqoop.connector.jdbc;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
import org.apache.sqoop.job.etl.Loader;
import org.apache.sqoop.job.etl.LoaderContext;
public class GenericJdbcExportLoader extends Loader<ConnectionConfiguration, ExportJobConfiguration> {
public class GenericJdbcLoader extends Loader<ConnectionConfiguration, ToJobConfiguration> {
public static final int DEFAULT_ROWS_PER_BATCH = 100;
public static final int DEFAULT_BATCHES_PER_TRANSACTION = 100;
@ -30,7 +30,7 @@ public class GenericJdbcExportLoader extends Loader<ConnectionConfiguration, Exp
private int batchesPerTransaction = DEFAULT_BATCHES_PER_TRANSACTION;
@Override
public void load(LoaderContext context, ConnectionConfiguration connection, ExportJobConfiguration job) throws Exception{
public void load(LoaderContext context, ConnectionConfiguration connection, ToJobConfiguration job) throws Exception{
String driver = connection.connection.jdbcDriver;
String url = connection.connection.connectionString;
String username = connection.connection.username;
@ -38,7 +38,7 @@ public class GenericJdbcExportLoader extends Loader<ConnectionConfiguration, Exp
GenericJdbcExecutor executor = new GenericJdbcExecutor(driver, url, username, password);
executor.setAutoCommit(false);
String sql = context.getString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_DATA_SQL);
String sql = context.getString(GenericJdbcConnectorConstants.CONNECTOR_TO_JDBC_DATA_SQL);
executor.beginBatch(sql);
try {
int numberOfRows = 0;

Просмотреть файл

@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.sqoop.job.etl.Partition;
public class GenericJdbcImportPartition extends Partition {
public class GenericJdbcPartition extends Partition {
private String conditions;

Просмотреть файл

@ -22,7 +22,6 @@ import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
import java.sql.Types;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.LinkedList;
import java.util.List;
@ -30,12 +29,12 @@ import java.util.TimeZone;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
import org.apache.sqoop.job.etl.Partition;
import org.apache.sqoop.job.etl.Partitioner;
import org.apache.sqoop.job.etl.PartitionerContext;
public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfiguration, ImportJobConfiguration> {
public class GenericJdbcPartitioner extends Partitioner<ConnectionConfiguration, FromJobConfiguration> {
private static final BigDecimal NUMERIC_MIN_INCREMENT = new BigDecimal(10000 * Double.MIN_VALUE);
@ -48,7 +47,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
private Boolean partitionColumnNull;
@Override
public List<Partition> getPartitions(PartitionerContext context,ConnectionConfiguration connection, ImportJobConfiguration job) {
public List<Partition> getPartitions(PartitionerContext context,ConnectionConfiguration connection, FromJobConfiguration job) {
List<Partition> partitions = new LinkedList<Partition>();
numberPartitions = context.getMaxPartitions();
@ -63,14 +62,14 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
}
if (partitionMinValue == null && partitionMaxValue == null) {
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(partitionColumnName + " IS NULL");
partitions.add(partition);
return partitions;
}
if (partitionColumnNull) {
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(partitionColumnName + " IS NULL");
partitions.add(partition);
numberPartitions -= 1;
@ -190,7 +189,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
break;
}
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(
constructDateConditions(sdf, objLB, objUB, false));
partitions.add(partition);
@ -212,7 +211,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
}
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(
constructDateConditions(sdf, objLB, objUB, true));
partitions.add(partition);
@ -249,7 +248,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
// Having one single value means that we can create only one single split
if(minStringBD.equals(maxStringBD)) {
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(constructTextConditions(prefix, 0, 0,
partitionMinValue, partitionMaxValue, true, true));
partitions.add(partition);
@ -294,7 +293,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
for (int i = 1; i < splitPoints.size(); i++) {
BigDecimal end = splitPoints.get(i);
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(constructTextConditions(prefix, start, end,
partitionMinValue, partitionMaxValue, i == 1, i == splitPoints.size() - 1));
partitions.add(partition);
@ -327,13 +326,13 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
upperBound = lowerBound + interval;
upperBound += (i <= remainder) ? 1 : 0;
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(
constructConditions(lowerBound, upperBound, false));
partitions.add(partition);
}
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(
constructConditions(upperBound, maxValue, true));
partitions.add(partition);
@ -357,13 +356,13 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
lowerBound = upperBound;
upperBound = lowerBound + interval;
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(
constructConditions(lowerBound, upperBound, false));
partitions.add(partition);
}
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(
constructConditions(upperBound, maxValue, true));
partitions.add(partition);
@ -383,7 +382,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
// Having one single value means that we can create only one single split
if(minValue.equals(maxValue)) {
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(constructConditions(minValue));
partitions.add(partition);
return partitions;
@ -415,7 +414,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
for (int i = 1; i < splitPoints.size(); i++) {
BigDecimal end = splitPoints.get(i);
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
partition.setConditions(constructConditions(start, end, i == splitPoints.size() - 1));
partitions.add(partition);
@ -436,7 +435,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
// Having one single value means that we can create only one single split
if(minValue.equals(maxValue)) {
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
conditions.append(partitionColumnName).append(" = ")
.append(maxValue);
@ -445,7 +444,7 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
return partitions;
}
GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
GenericJdbcPartition partition = new GenericJdbcPartition();
if (partitionMinValue == null) {
conditions = new StringBuilder();
@ -453,12 +452,12 @@ public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfigur
partition.setConditions(conditions.toString());
partitions.add(partition);
}
partition = new GenericJdbcImportPartition();
partition = new GenericJdbcPartition();
conditions = new StringBuilder();
conditions.append(partitionColumnName).append(" = TRUE");
partition.setConditions(conditions.toString());
partitions.add(partition);
partition = new GenericJdbcImportPartition();
partition = new GenericJdbcPartition();
conditions = new StringBuilder();
conditions.append(partitionColumnName).append(" = FALSE");
partition.setConditions(conditions.toString());

Просмотреть файл

@ -19,16 +19,16 @@ package org.apache.sqoop.connector.jdbc;
import org.apache.log4j.Logger;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
import org.apache.sqoop.job.etl.Destroyer;
import org.apache.sqoop.job.etl.DestroyerContext;
public class GenericJdbcExportDestroyer extends Destroyer<ConnectionConfiguration, ExportJobConfiguration> {
public class GenericJdbcToDestroyer extends Destroyer<ConnectionConfiguration, ToJobConfiguration> {
private static final Logger LOG = Logger.getLogger(GenericJdbcExportDestroyer.class);
private static final Logger LOG = Logger.getLogger(GenericJdbcToDestroyer.class);
@Override
public void destroy(DestroyerContext context, ConnectionConfiguration connection, ExportJobConfiguration job) {
public void destroy(DestroyerContext context, ConnectionConfiguration connection, ToJobConfiguration job) {
LOG.info("Running generic JDBC connector destroyer");
final String tableName = job.table.tableName;

Просмотреть файл

@ -28,7 +28,7 @@ import org.apache.log4j.Logger;
import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
import org.apache.sqoop.connector.jdbc.util.SqlTypesUtils;
import org.apache.sqoop.job.etl.Initializer;
import org.apache.sqoop.job.etl.InitializerContext;
@ -36,14 +36,14 @@ import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.schema.type.Column;
import org.apache.sqoop.utils.ClassUtils;
public class GenericJdbcExportInitializer extends Initializer<ConnectionConfiguration, ExportJobConfiguration> {
public class GenericJdbcToInitializer extends Initializer<ConnectionConfiguration, ToJobConfiguration> {
private GenericJdbcExecutor executor;
private static final Logger LOG =
Logger.getLogger(GenericJdbcExportInitializer.class);
Logger.getLogger(GenericJdbcToInitializer.class);
@Override
public void initialize(InitializerContext context, ConnectionConfiguration connection, ExportJobConfiguration job) {
public void initialize(InitializerContext context, ConnectionConfiguration connection, ToJobConfiguration job) {
configureJdbcProperties(context.getContext(), connection, job);
try {
configureTableProperties(context.getContext(), connection, job);
@ -53,7 +53,7 @@ public class GenericJdbcExportInitializer extends Initializer<ConnectionConfigur
}
@Override
public List<String> getJars(InitializerContext context, ConnectionConfiguration connection, ExportJobConfiguration job) {
public List<String> getJars(InitializerContext context, ConnectionConfiguration connection, ToJobConfiguration job) {
List<String> jars = new LinkedList<String>();
jars.add(ClassUtils.jarForClass(connection.connection.jdbcDriver));
@ -62,18 +62,18 @@ public class GenericJdbcExportInitializer extends Initializer<ConnectionConfigur
}
@Override
public Schema getSchema(InitializerContext context, ConnectionConfiguration connectionConfiguration, ExportJobConfiguration exportJobConfiguration) {
configureJdbcProperties(context.getContext(), connectionConfiguration, exportJobConfiguration);
public Schema getSchema(InitializerContext context, ConnectionConfiguration connectionConfiguration, ToJobConfiguration toJobConfiguration) {
configureJdbcProperties(context.getContext(), connectionConfiguration, toJobConfiguration);
String schemaName = exportJobConfiguration.table.tableName;
String schemaName = toJobConfiguration.table.tableName;
if (schemaName == null) {
throw new SqoopException(GenericJdbcConnectorError.GENERIC_JDBC_CONNECTOR_0019,
"Table name extraction not supported yet.");
}
if(exportJobConfiguration.table.schemaName != null) {
schemaName = exportJobConfiguration.table.schemaName + "." + schemaName;
if(toJobConfiguration.table.schemaName != null) {
schemaName = toJobConfiguration.table.schemaName + "." + schemaName;
}
Schema schema = new Schema(schemaName);
@ -112,7 +112,7 @@ public class GenericJdbcExportInitializer extends Initializer<ConnectionConfigur
}
}
private void configureJdbcProperties(MutableContext context, ConnectionConfiguration connectionConfig, ExportJobConfiguration jobConfig) {
private void configureJdbcProperties(MutableContext context, ConnectionConfiguration connectionConfig, ToJobConfiguration jobConfig) {
String driver = connectionConfig.connection.jdbcDriver;
String url = connectionConfig.connection.connectionString;
String username = connectionConfig.connection.username;
@ -124,7 +124,7 @@ public class GenericJdbcExportInitializer extends Initializer<ConnectionConfigur
executor = new GenericJdbcExecutor(driver, url, username, password);
}
private void configureTableProperties(MutableContext context, ConnectionConfiguration connectionConfig, ExportJobConfiguration jobConfig) {
private void configureTableProperties(MutableContext context, ConnectionConfiguration connectionConfig, ToJobConfiguration jobConfig) {
String dataSql;
String schemaName = jobConfig.table.schemaName;
@ -216,7 +216,7 @@ public class GenericJdbcExportInitializer extends Initializer<ConnectionConfigur
GenericJdbcConnectorError.GENERIC_JDBC_CONNECTOR_0008);
}
context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_DATA_SQL,
dataSql.toString());
context.setString(GenericJdbcConnectorConstants.CONNECTOR_TO_JDBC_DATA_SQL,
dataSql);
}
}

Просмотреть файл

@ -18,9 +18,8 @@
package org.apache.sqoop.connector.jdbc;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
import org.apache.sqoop.validation.Status;
import org.apache.sqoop.validation.Validation;
import org.apache.sqoop.validation.Validator;
@ -67,20 +66,13 @@ public class GenericJdbcValidator extends Validator {
}
@Override
public Validation validateJob(MJob.Type type, Object jobConfiguration) {
switch(type) {
case IMPORT:
return validateImportJob(jobConfiguration);
case EXPORT:
return validateExportJob(jobConfiguration);
default:
return super.validateJob(type, jobConfiguration);
}
public Validation validateJob(Object jobConfiguration) {
return super.validateJob(jobConfiguration);
}
private Validation validateExportJob(Object jobConfiguration) {
Validation validation = new Validation(ExportJobConfiguration.class);
ExportJobConfiguration configuration = (ExportJobConfiguration)jobConfiguration;
Validation validation = new Validation(ToJobConfiguration.class);
ToJobConfiguration configuration = (ToJobConfiguration)jobConfiguration;
if(configuration.table.tableName == null && configuration.table.sql == null) {
validation.addMessage(Status.UNACCEPTABLE, "table", "Either table name or SQL must be specified");
@ -104,8 +96,8 @@ public class GenericJdbcValidator extends Validator {
}
private Validation validateImportJob(Object jobConfiguration) {
Validation validation = new Validation(ImportJobConfiguration.class);
ImportJobConfiguration configuration = (ImportJobConfiguration)jobConfiguration;
Validation validation = new Validation(FromJobConfiguration.class);
FromJobConfiguration configuration = (FromJobConfiguration)jobConfiguration;
if(configuration.table.tableName == null && configuration.table.sql == null) {
validation.addMessage(Status.UNACCEPTABLE, "table", "Either table name or SQL must be specified");

Просмотреть файл

@ -24,10 +24,10 @@ import org.apache.sqoop.model.Form;
*
*/
@ConfigurationClass
public class ImportJobConfiguration {
@Form public ImportTableForm table;
public class FromJobConfiguration {
@Form public FromTableForm table;
public ImportJobConfiguration() {
table = new ImportTableForm();
public FromJobConfiguration() {
table = new FromTableForm();
}
}

Просмотреть файл

@ -24,7 +24,7 @@ import org.apache.sqoop.model.Input;
*
*/
@FormClass
public class ImportTableForm {
public class FromTableForm {
@Input(size = 50) public String schemaName;
@Input(size = 50) public String tableName;
@Input(size = 2000) public String sql;

Просмотреть файл

@ -24,10 +24,10 @@ import org.apache.sqoop.model.Form;
*
*/
@ConfigurationClass
public class ExportJobConfiguration {
@Form public ExportTableForm table;
public class ToJobConfiguration {
@Form public ToTableForm table;
public ExportJobConfiguration() {
table = new ExportTableForm();
public ToJobConfiguration() {
table = new ToTableForm();
}
}

Просмотреть файл

@ -24,7 +24,7 @@ import org.apache.sqoop.model.Input;
*
*/
@FormClass
public class ExportTableForm {
public class ToTableForm {
@Input(size = 50) public String schemaName;
@Input(size = 2000) public String tableName;
@Input(size = 50) public String sql;

Просмотреть файл

@ -22,7 +22,7 @@ import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
//import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
import org.apache.sqoop.job.etl.Initializer;
import org.apache.sqoop.job.etl.InitializerContext;
import org.apache.sqoop.model.MJob;

Просмотреть файл

@ -27,7 +27,7 @@ import java.util.Collection;
import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
//import org.apache.sqoop.connector.jdbc.configuration.ExportJobConfiguration;
import org.apache.sqoop.etl.io.DataReader;
import org.apache.sqoop.job.etl.Loader;
import org.apache.sqoop.job.etl.LoaderContext;

Просмотреть файл

@ -22,7 +22,7 @@ import junit.framework.TestCase;
import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
//import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.job.etl.Extractor;
import org.apache.sqoop.job.etl.ExtractorContext;
import org.apache.sqoop.etl.io.DataWriter;

Просмотреть файл

@ -24,7 +24,7 @@ import junit.framework.TestCase;
import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
//import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.job.Constants;
import org.apache.sqoop.job.etl.Initializer;
import org.apache.sqoop.job.etl.InitializerContext;

Просмотреть файл

@ -31,7 +31,7 @@ import org.apache.sqoop.common.MutableContext;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
//import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
import org.apache.sqoop.job.Constants;
import org.apache.sqoop.job.etl.Partition;
import org.apache.sqoop.job.etl.Partitioner;

Просмотреть файл

@ -23,8 +23,8 @@ import java.util.List;
import java.util.Locale;
import java.util.ResourceBundle;
import org.apache.sqoop.job.etl.Exporter;
import org.apache.sqoop.job.etl.Importer;
import org.apache.sqoop.job.etl.From;
import org.apache.sqoop.job.etl.To;
import org.apache.sqoop.model.MConnectionForms;
import org.apache.sqoop.model.MForm;
import org.apache.sqoop.connector.spi.SqoopConnector;
@ -53,13 +53,13 @@ public class MySqlJdbcConnector implements SqoopConnector {
}
@Override
public Importer getImporter() {
public From getImporter() {
// TODO Auto-generated method stub
return null;
}
@Override
public Exporter getExporter() {
public To getExporter() {
// TODO Auto-generated method stub
return null;
}

Просмотреть файл

@ -19,18 +19,16 @@ package org.apache.sqoop.connector;
import java.io.IOException;
import java.net.URL;
import java.util.LinkedList;
import java.util.List;
import java.util.Properties;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.core.ConfigurationConstants;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MConnectionForms;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.spi.SqoopConnector;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MJobForms;
public final class ConnectorHandler {
@ -93,21 +91,19 @@ public final class ConnectorHandler {
}
// Initialize Metadata
List<MJobForms> jobForms = new LinkedList<MJobForms>();
for(MJob.Type type : MJob.Type.values()) {
Class klass = connector.getJobConfigurationClass(type);
if(klass != null) {
jobForms.add(new MJobForms(type, FormUtils.toForms(klass)));
}
}
MJobForms fromJobForms = new MJobForms(FormUtils.toForms(
connector.getJobConfigurationClass(ConnectorType.FROM)));
MConnectionForms connectionForms = new MConnectionForms(
FormUtils.toForms(connector.getConnectionConfigurationClass()));
MJobForms toJobForms = new MJobForms(FormUtils.toForms(
connector.getJobConfigurationClass(ConnectorType.TO)));
MConnectionForms toConnectionForms = new MConnectionForms(
FormUtils.toForms(connector.getConnectionConfigurationClass()));
String connectorVersion = connector.getVersion();
mConnector = new MConnector(connectorUniqueName, connectorClassName,
connectorVersion, connectionForms, jobForms);
mConnector = new MConnector(connectorUniqueName, connectorClassName, connectorVersion,
connectionForms, fromJobForms, toJobForms);
if (LOG.isInfoEnabled()) {
LOG.info("Connector [" + connectorClassName + "] initialized.");

Просмотреть файл

@ -52,15 +52,9 @@ public abstract class ExecutionEngine {
}
/**
* Prepare given submission request for import job type.
* Prepare given submission request.
*
* @param request Submission request
*/
public abstract void prepareImportSubmission(SubmissionRequest request);
/**
* Prepare given submission request for export job type..
* @param request
*/
public abstract void prepareExportSubmission(SubmissionRequest request);
public abstract void prepareSubmission(SubmissionRequest request);
}

Просмотреть файл

@ -24,14 +24,11 @@ import org.apache.sqoop.core.Reconfigurable;
import org.apache.sqoop.core.SqoopConfiguration;
import org.apache.sqoop.core.SqoopConfiguration.CoreConfigurationListener;
import org.apache.sqoop.framework.configuration.ConnectionConfiguration;
import org.apache.sqoop.framework.configuration.ExportJobConfiguration;
import org.apache.sqoop.framework.configuration.ImportJobConfiguration;
import org.apache.sqoop.framework.configuration.JobConfiguration;
import org.apache.sqoop.model.*;
import org.apache.sqoop.repository.RepositoryManager;
import org.apache.sqoop.validation.Validator;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.ResourceBundle;
@ -113,31 +110,20 @@ public class FrameworkManager implements Reconfigurable {
public static final String CURRENT_FRAMEWORK_VERSION = "1";
public Class getJobConfigurationClass(MJob.Type jobType) {
switch (jobType) {
case IMPORT:
return ImportJobConfiguration.class;
case EXPORT:
return ExportJobConfiguration.class;
default:
return null;
}
public Class getJobConfigurationClass() {
return JobConfiguration.class;
}
public Class getConnectionConfigurationClass() {
return ConnectionConfiguration.class;
}
public Class getConnectionConfigurationClass() {
return ConnectionConfiguration.class;
}
public FrameworkManager() {
MConnectionForms connectionForms = new MConnectionForms(
FormUtils.toForms(getConnectionConfigurationClass())
);
List<MJobForms> jobForms = new LinkedList<MJobForms>();
jobForms.add(new MJobForms(MJob.Type.IMPORT,
FormUtils.toForms(getJobConfigurationClass(MJob.Type.IMPORT))));
jobForms.add(new MJobForms(MJob.Type.EXPORT,
FormUtils.toForms(getJobConfigurationClass(MJob.Type.EXPORT))));
mFramework = new MFramework(connectionForms, jobForms,
CURRENT_FRAMEWORK_VERSION);
mFramework = new MFramework(connectionForms, new MJobForms(FormUtils.toForms(getJobConfigurationClass())),
CURRENT_FRAMEWORK_VERSION);
// Build validator
validator = new FrameworkValidator();

Просмотреть файл

@ -18,13 +18,11 @@
package org.apache.sqoop.framework;
import org.apache.sqoop.framework.configuration.ConnectionConfiguration;
import org.apache.sqoop.framework.configuration.ExportJobConfiguration;
import org.apache.sqoop.framework.configuration.ImportJobConfiguration;
import org.apache.sqoop.framework.configuration.InputForm;
import org.apache.sqoop.framework.configuration.JobConfiguration;
import org.apache.sqoop.framework.configuration.OutputCompression;
import org.apache.sqoop.framework.configuration.OutputForm;
import org.apache.sqoop.framework.configuration.ThrottlingForm;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.validation.Status;
import org.apache.sqoop.validation.Validation;
import org.apache.sqoop.validation.Validator;
@ -43,61 +41,57 @@ public class FrameworkValidator extends Validator {
@Override
public Validation validateJob(MJob.Type type, Object jobConfiguration) {
switch(type) {
case IMPORT:
return validateImportJob(jobConfiguration);
case EXPORT:
return validateExportJob(jobConfiguration);
default:
return super.validateJob(type, jobConfiguration);
}
}
private Validation validateExportJob(Object jobConfiguration) {
Validation validation = new Validation(ExportJobConfiguration.class);
ExportJobConfiguration configuration = (ExportJobConfiguration)jobConfiguration;
validateInputForm(validation, configuration.input);
public Validation validateJob(Object jobConfiguration) {
JobConfiguration configuration = (JobConfiguration)jobConfiguration;
Validation validation = new Validation(JobConfiguration.class);
validateThrottingForm(validation, configuration.throttling);
return validation;
return super.validateJob(jobConfiguration);
}
private Validation validateImportJob(Object jobConfiguration) {
Validation validation = new Validation(ImportJobConfiguration.class);
ImportJobConfiguration configuration = (ImportJobConfiguration)jobConfiguration;
// private Validation validateExportJob(Object jobConfiguration) {
// Validation validation = new Validation(ExportJobConfiguration.class);
// ExportJobConfiguration configuration = (ExportJobConfiguration)jobConfiguration;
//
// validateInputForm(validation, configuration.input);
// validateThrottingForm(validation, configuration.throttling);
//
// return validation;
// }
//
// private Validation validateImportJob(Object jobConfiguration) {
// Validation validation = new Validation(ImportJobConfiguration.class);
// ImportJobConfiguration configuration = (ImportJobConfiguration)jobConfiguration;
//
// validateOutputForm(validation, configuration.output);
// validateThrottingForm(validation, configuration.throttling);
//
// return validation;
// }
validateOutputForm(validation, configuration.output);
validateThrottingForm(validation, configuration.throttling);
return validation;
}
private void validateInputForm(Validation validation, InputForm input) {
if(input.inputDirectory == null || input.inputDirectory.isEmpty()) {
validation.addMessage(Status.UNACCEPTABLE, "input", "inputDirectory", "Input directory is empty");
}
}
private void validateOutputForm(Validation validation, OutputForm output) {
if(output.outputDirectory == null || output.outputDirectory.isEmpty()) {
validation.addMessage(Status.UNACCEPTABLE, "output", "outputDirectory", "Output directory is empty");
}
if(output.customCompression != null &&
output.customCompression.trim().length() > 0 &&
output.compression != OutputCompression.CUSTOM) {
validation.addMessage(Status.UNACCEPTABLE, "output", "compression",
"custom compression should be blank as " + output.compression + " is being used.");
}
if(output.compression == OutputCompression.CUSTOM &&
(output.customCompression == null ||
output.customCompression.trim().length() == 0)
) {
validation.addMessage(Status.UNACCEPTABLE, "output", "compression",
"custom compression is blank.");
}
}
// private void validateInputForm(Validation validation, InputForm input) {
// if(input.inputDirectory == null || input.inputDirectory.isEmpty()) {
// validation.addMessage(Status.UNACCEPTABLE, "input", "inputDirectory", "Input directory is empty");
// }
// }
//
// private void validateOutputForm(Validation validation, OutputForm output) {
// if(output.outputDirectory == null || output.outputDirectory.isEmpty()) {
// validation.addMessage(Status.UNACCEPTABLE, "output", "outputDirectory", "Output directory is empty");
// }
// if(output.customCompression != null &&
// output.customCompression.trim().length() > 0 &&
// output.compression != OutputCompression.CUSTOM) {
// validation.addMessage(Status.UNACCEPTABLE, "output", "compression",
// "custom compression should be blank as " + output.compression + " is being used.");
// }
// if(output.compression == OutputCompression.CUSTOM &&
// (output.customCompression == null ||
// output.customCompression.trim().length() == 0)
// ) {
// validation.addMessage(Status.UNACCEPTABLE, "output", "compression",
// "custom compression is blank.");
// }
// }
private void validateThrottingForm(Validation validation, ThrottlingForm throttling) {
if(throttling.extractors != null && throttling.extractors < 1) {

Просмотреть файл

@ -18,17 +18,17 @@
package org.apache.sqoop.framework;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.MapContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.ConnectorManager;
import org.apache.sqoop.framework.configuration.JobConfiguration;
import org.apache.sqoop.request.HttpEventContext;
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
import org.apache.sqoop.connector.spi.SqoopConnector;
import org.apache.sqoop.core.Reconfigurable;
import org.apache.sqoop.core.SqoopConfiguration;
import org.apache.sqoop.core.SqoopConfiguration.CoreConfigurationListener;
import org.apache.sqoop.framework.configuration.ExportJobConfiguration;
import org.apache.sqoop.framework.configuration.ImportJobConfiguration;
import org.apache.sqoop.job.etl.*;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MConnection;
@ -280,34 +280,52 @@ public class JobManager implements Reconfigurable {
"Job id: " + job.getPersistenceId());
}
MConnection connection = repository.findConnection(job.getConnectionId());
MConnection fromConnection = repository.findConnection(job.getConnectionId(ConnectorType.FROM));
MConnection toConnection = repository.findConnection(job.getConnectionId(ConnectorType.TO));
if (!connection.getEnabled()) {
if (!fromConnection.getEnabled()) {
throw new SqoopException(FrameworkError.FRAMEWORK_0010,
"Connection id: " + connection.getPersistenceId());
"Connection id: " + fromConnection.getPersistenceId());
}
SqoopConnector connector =
ConnectorManager.getInstance().getConnector(job.getConnectorId());
SqoopConnector fromConnector =
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.FROM));
SqoopConnector toConnector =
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.TO));
// Transform forms to connector specific classes
Object connectorConnection = ClassUtils.instantiate(
connector.getConnectionConfigurationClass());
FormUtils.fromForms(connection.getConnectorPart().getForms(),
connectorConnection);
// Transform forms to fromConnector specific classes
Object fromConnectorConnection = ClassUtils.instantiate(
fromConnector.getConnectionConfigurationClass());
FormUtils.fromForms(fromConnection.getConnectorPart().getForms(),
fromConnectorConnection);
Object connectorJob = ClassUtils.instantiate(
connector.getJobConfigurationClass(job.getType()));
FormUtils.fromForms(job.getConnectorPart().getForms(), connectorJob);
Object fromJob = ClassUtils.instantiate(
fromConnector.getJobConfigurationClass(ConnectorType.FROM));
FormUtils.fromForms(
job.getConnectorPart(ConnectorType.FROM).getForms(), fromJob);
// Transform forms to toConnector specific classes
Object toConnectorConnection = ClassUtils.instantiate(
toConnector.getConnectionConfigurationClass());
FormUtils.fromForms(toConnection.getConnectorPart().getForms(),
toConnectorConnection);
Object toJob = ClassUtils.instantiate(
toConnector.getJobConfigurationClass(ConnectorType.TO));
FormUtils.fromForms(job.getConnectorPart(ConnectorType.TO).getForms(), toJob);
// Transform framework specific forms
Object frameworkConnection = ClassUtils.instantiate(
Object fromFrameworkConnection = ClassUtils.instantiate(
FrameworkManager.getInstance().getConnectionConfigurationClass());
FormUtils.fromForms(connection.getFrameworkPart().getForms(),
frameworkConnection);
Object toFrameworkConnection = ClassUtils.instantiate(
FrameworkManager.getInstance().getConnectionConfigurationClass());
FormUtils.fromForms(fromConnection.getFrameworkPart().getForms(),
fromFrameworkConnection);
FormUtils.fromForms(toConnection.getFrameworkPart().getForms(),
toFrameworkConnection);
Object frameworkJob = ClassUtils.instantiate(
FrameworkManager.getInstance().getJobConfigurationClass(job.getType()));
FrameworkManager.getInstance().getJobConfigurationClass());
FormUtils.fromForms(job.getFrameworkPart().getForms(), frameworkJob);
// Create request object
@ -319,12 +337,16 @@ public class JobManager implements Reconfigurable {
// Save important variables to the submission request
request.setSummary(summary);
request.setConnector(connector);
request.setConfigConnectorConnection(connectorConnection);
request.setConfigConnectorJob(connectorJob);
request.setConfigFrameworkConnection(frameworkConnection);
request.setConnector(ConnectorType.FROM, fromConnector);
request.setConnector(ConnectorType.TO, toConnector);
request.setConnectorConnectionConfig(ConnectorType.FROM, fromConnectorConnection);
request.setConnectorConnectionConfig(ConnectorType.TO, toConnectorConnection);
request.setConnectorJobConfig(ConnectorType.FROM, fromJob);
request.setConnectorJobConfig(ConnectorType.TO, toJob);
// @TODO(Abe): Should we actually have 2 different Framework Connection config objects?
request.setFrameworkConnectionConfig(ConnectorType.FROM, fromFrameworkConnection);
request.setFrameworkConnectionConfig(ConnectorType.TO, toFrameworkConnection);
request.setConfigFrameworkJob(frameworkJob);
request.setJobType(job.getType());
request.setJobName(job.getName());
request.setJobId(job.getPersistenceId());
request.setNotificationUrl(notificationBaseUrl + jobId);
@ -342,8 +364,9 @@ public class JobManager implements Reconfigurable {
request.addJarForClass(SqoopConnector.class);
// Execution engine jar
request.addJarForClass(executionEngine.getClass());
// Connector in use
request.addJarForClass(connector.getClass());
// Connectors in use
request.addJarForClass(fromConnector.getClass());
request.addJarForClass(toConnector.getClass());
// Extra libraries that Sqoop code requires
request.addJarForClass(JSONValue.class);
@ -351,66 +374,93 @@ public class JobManager implements Reconfigurable {
// The IDF is used in the ETL process.
request.addJarForClass(dataFormatClass);
// Get connector callbacks
switch (job.getType()) {
case IMPORT:
request.setConnectorCallbacks(connector.getImporter());
break;
case EXPORT:
request.setConnectorCallbacks(connector.getExporter());
break;
default:
throw new SqoopException(FrameworkError.FRAMEWORK_0005,
"Unsupported job type " + job.getType().name());
}
LOG.debug("Using callbacks: " + request.getConnectorCallbacks());
// Initialize submission from connector perspective
CallbackBase baseCallbacks = request.getConnectorCallbacks();
// Get callbacks
request.setFromCallback(fromConnector.getFrom());
request.setToCallback(toConnector.getTo());
LOG.debug("Using callbacks: " + request.getFromCallback() + ", " + request.getToCallback());
Class<? extends Initializer> initializerClass = baseCallbacks
.getInitializer();
Initializer initializer = (Initializer) ClassUtils
.instantiate(initializerClass);
// Initialize submission from fromConnector perspective
CallbackBase[] baseCallbacks = {
request.getFromCallback(),
request.getToCallback()
};
CallbackBase baseCallback;
Class<? extends Initializer> initializerClass;
Initializer initializer;
InitializerContext initializerContext;
// Initialize From Connector callback.
baseCallback = request.getFromCallback();
initializerClass = baseCallback
.getInitializer();
initializer = (Initializer) ClassUtils
.instantiate(initializerClass);
if (initializer == null) {
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
"Can't create initializer instance: " + initializerClass.getName());
"Can't create initializer instance: " + initializerClass.getName());
}
// Initializer context
InitializerContext initializerContext = new InitializerContext(
request.getConnectorContext());
initializerContext = new InitializerContext(request.getConnectorContext(ConnectorType.FROM));
// Initialize submission from connector perspective
// Initialize submission from fromConnector perspective
initializer.initialize(initializerContext,
request.getConfigConnectorConnection(),
request.getConfigConnectorJob());
request.getConnectorConnectionConfig(ConnectorType.FROM),
request.getConnectorJobConfig(ConnectorType.FROM));
// Add job specific jars to
request.addJars(initializer.getJars(initializerContext,
request.getConfigConnectorConnection(),
request.getConfigConnectorJob()));
request.getConnectorConnectionConfig(ConnectorType.FROM),
request.getConnectorJobConfig(ConnectorType.FROM)));
// @TODO(Abe): Alter behavior of Schema here. Need from Schema.
// Retrieve and persist the schema
request.getSummary().setConnectorSchema(initializer.getSchema(
initializerContext,
request.getConfigConnectorConnection(),
request.getConfigConnectorJob()
));
initializerContext,
request.getConnectorConnectionConfig(ConnectorType.FROM),
request.getConnectorJobConfig(ConnectorType.FROM)
));
// Initialize To Connector callback.
baseCallback = request.getToCallback();
initializerClass = baseCallback
.getInitializer();
initializer = (Initializer) ClassUtils
.instantiate(initializerClass);
if (initializer == null) {
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
"Can't create initializer instance: " + initializerClass.getName());
}
// Initializer context
initializerContext = new InitializerContext(request.getConnectorContext(ConnectorType.TO));
// Initialize submission from fromConnector perspective
initializer.initialize(initializerContext,
request.getConnectorConnectionConfig(ConnectorType.TO),
request.getConnectorJobConfig(ConnectorType.TO));
// Add job specific jars to
request.addJars(initializer.getJars(initializerContext,
request.getConnectorConnectionConfig(ConnectorType.TO),
request.getConnectorJobConfig(ConnectorType.TO)));
// @TODO(Abe): Alter behavior of Schema here. Need To Schema.
// Retrieve and persist the schema
// request.getSummary().setConnectorSchema(initializer.getSchema(
// initializerContext,
// request.getConnectorConnectionConfig(ConnectorType.TO),
// request.getConnectorJobConfig(ConnectorType.TO)
// ));
// Bootstrap job from framework perspective
switch (job.getType()) {
case IMPORT:
prepareImportSubmission(request);
break;
case EXPORT:
prepareExportSubmission(request);
break;
default:
throw new SqoopException(FrameworkError.FRAMEWORK_0005,
"Unsupported job type " + job.getType().name());
}
prepareSubmission(request);
// Make sure that this job id is not currently running and submit the job
// only if it's not.
@ -421,6 +471,7 @@ public class JobManager implements Reconfigurable {
"Job with id " + jobId);
}
// @TODO(Abe): Call multiple destroyers.
// TODO(jarcec): We might need to catch all exceptions here to ensure
// that Destroyer will be executed in all cases.
boolean submitted = submissionEngine.submit(request);
@ -436,12 +487,9 @@ public class JobManager implements Reconfigurable {
return summary;
}
private void prepareImportSubmission(SubmissionRequest request) {
ImportJobConfiguration jobConfiguration = (ImportJobConfiguration) request
.getConfigFrameworkJob();
// Initialize the map-reduce part (all sort of required classes, ...)
request.setOutputDirectory(jobConfiguration.output.outputDirectory);
private void prepareSubmission(SubmissionRequest request) {
JobConfiguration jobConfiguration = (JobConfiguration) request
.getConfigFrameworkJob();
// We're directly moving configured number of extractors and loaders to
// underlying request object. In the future we might need to throttle this
@ -450,21 +498,7 @@ public class JobManager implements Reconfigurable {
request.setLoaders(jobConfiguration.throttling.loaders);
// Delegate rest of the job to execution engine
executionEngine.prepareImportSubmission(request);
}
private void prepareExportSubmission(SubmissionRequest request) {
ExportJobConfiguration jobConfiguration = (ExportJobConfiguration) request
.getConfigFrameworkJob();
// We're directly moving configured number of extractors and loaders to
// underlying request object. In the future we might need to throttle this
// count based on other running jobs to meet our SLAs.
request.setExtractors(jobConfiguration.throttling.extractors);
request.setLoaders(jobConfiguration.throttling.loaders);
// Delegate rest of the job to execution engine
executionEngine.prepareExportSubmission(request);
executionEngine.prepareSubmission(request);
}
/**
@ -472,23 +506,37 @@ public class JobManager implements Reconfigurable {
* remote cluster.
*/
private void destroySubmission(SubmissionRequest request) {
CallbackBase baseCallbacks = request.getConnectorCallbacks();
CallbackBase fromCallback = request.getFromCallback();
CallbackBase toCallback = request.getToCallback();
Class<? extends Destroyer> destroyerClass = baseCallbacks.getDestroyer();
Destroyer destroyer = (Destroyer) ClassUtils.instantiate(destroyerClass);
Class<? extends Destroyer> fromDestroyerClass = fromCallback.getDestroyer();
Class<? extends Destroyer> toDestroyerClass = toCallback.getDestroyer();
Destroyer fromDestroyer = (Destroyer) ClassUtils.instantiate(fromDestroyerClass);
Destroyer toDestroyer = (Destroyer) ClassUtils.instantiate(toDestroyerClass);
if (destroyer == null) {
if (fromDestroyer == null) {
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
"Can't create destroyer instance: " + destroyerClass.getName());
"Can't create toDestroyer instance: " + fromDestroyerClass.getName());
}
DestroyerContext destroyerContext = new DestroyerContext(
request.getConnectorContext(), false, request.getSummary()
if (toDestroyer == null) {
throw new SqoopException(FrameworkError.FRAMEWORK_0006,
"Can't create toDestroyer instance: " + toDestroyerClass.getName());
}
// @TODO(Abe): Update context to manage multiple connectors. As well as summary.
DestroyerContext fromDestroyerContext = new DestroyerContext(
request.getConnectorContext(ConnectorType.FROM), false, request.getSummary()
.getConnectorSchema());
DestroyerContext toDestroyerContext = new DestroyerContext(
request.getConnectorContext(ConnectorType.TO), false, request.getSummary()
.getConnectorSchema());
// Initialize submission from connector perspective
destroyer.destroy(destroyerContext, request.getConfigConnectorConnection(),
request.getConfigConnectorJob());
fromDestroyer.destroy(fromDestroyerContext, request.getConnectorConnectionConfig(ConnectorType.FROM),
request.getConnectorJobConfig(ConnectorType.FROM));
toDestroyer.destroy(toDestroyerContext, request.getConnectorConnectionConfig(ConnectorType.TO),
request.getConnectorJobConfig(ConnectorType.TO));
}
public MSubmission stop(long jobId, HttpEventContext ctx) {

Просмотреть файл

@ -17,16 +17,18 @@
*/
package org.apache.sqoop.framework;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
import org.apache.sqoop.connector.spi.SqoopConnector;
import org.apache.sqoop.job.etl.CallbackBase;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MSubmission;
import org.apache.sqoop.utils.ClassUtils;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
/**
* Submission details class is used when creating new submission and contains
@ -50,15 +52,10 @@ public class SubmissionRequest {
*/
long jobId;
/**
* Job type
*/
MJob.Type jobType;
/**
* Connector instance associated with this submission request
*/
SqoopConnector connector;
Map<ConnectorType, SqoopConnector > connectors;
/**
* List of required local jars for the job
@ -66,22 +63,27 @@ public class SubmissionRequest {
List<String> jars;
/**
* Base callbacks that are independent on job type
* From connector callback
*/
CallbackBase connectorCallbacks;
CallbackBase fromCallback;
/**
* All 4 configuration objects
* To connector callback
*/
Object configConnectorConnection;
Object configConnectorJob;
Object configFrameworkConnection;
CallbackBase toCallback;
/**
* All configuration objects
*/
Map<ConnectorType, Object> connectorConnectionConfigs;
Map<ConnectorType, Object> connectorJobConfigs;
Map<ConnectorType, Object> frameworkConnectionConfigs;
Object configFrameworkJob;
/**
* Connector context (submission specific configuration)
*/
MutableMapContext connectorContext;
Map<ConnectorType, MutableMapContext> connectorContexts;
/**
* Framework context (submission specific configuration)
@ -115,8 +117,17 @@ public class SubmissionRequest {
public SubmissionRequest() {
this.jars = new LinkedList<String>();
this.connectorContext = new MutableMapContext();
this.connectorContexts = new HashMap<ConnectorType, MutableMapContext>();
this.connectorContexts.put(ConnectorType.FROM, new MutableMapContext());
this.connectorContexts.put(ConnectorType.TO, new MutableMapContext());
this.frameworkContext = new MutableMapContext();
this.connectorConnectionConfigs = new HashMap<ConnectorType, Object>();
this.connectorJobConfigs = new HashMap<ConnectorType, Object>();
this.frameworkConnectionConfigs = new HashMap<ConnectorType, Object>();
this.connectors = new HashMap<ConnectorType, SqoopConnector>();
}
public MSubmission getSummary() {
@ -143,20 +154,12 @@ public class SubmissionRequest {
this.jobId = jobId;
}
public MJob.Type getJobType() {
return jobType;
public SqoopConnector getConnector(ConnectorType type) {
return connectors.get(type);
}
public void setJobType(MJob.Type jobType) {
this.jobType = jobType;
}
public SqoopConnector getConnector() {
return connector;
}
public void setConnector(SqoopConnector connector) {
this.connector = connector;
public void setConnector(ConnectorType type, SqoopConnector connector) {
this.connectors.put(type, connector);
}
public List<String> getJars() {
@ -179,36 +182,44 @@ public class SubmissionRequest {
}
}
public CallbackBase getConnectorCallbacks() {
return connectorCallbacks;
public CallbackBase getFromCallback() {
return fromCallback;
}
public void setConnectorCallbacks(CallbackBase connectorCallbacks) {
this.connectorCallbacks = connectorCallbacks;
public void setFromCallback(CallbackBase fromCallback) {
this.fromCallback = fromCallback;
}
public Object getConfigConnectorConnection() {
return configConnectorConnection;
public CallbackBase getToCallback() {
return toCallback;
}
public void setConfigConnectorConnection(Object config) {
configConnectorConnection = config;
public void setToCallback(CallbackBase toCallback) {
this.toCallback = toCallback;
}
public Object getConfigConnectorJob() {
return configConnectorJob;
public Object getConnectorConnectionConfig(ConnectorType type) {
return connectorConnectionConfigs.get(type);
}
public void setConfigConnectorJob(Object config) {
configConnectorJob = config;
public void setConnectorConnectionConfig(ConnectorType type, Object config) {
connectorConnectionConfigs.put(type, config);
}
public Object getConfigFrameworkConnection() {
return configFrameworkConnection;
public Object getConnectorJobConfig(ConnectorType type) {
return connectorJobConfigs.get(type);
}
public void setConfigFrameworkConnection(Object config) {
configFrameworkConnection = config;
public void setConnectorJobConfig(ConnectorType type, Object config) {
connectorJobConfigs.put(type, config);
}
public Object getFrameworkConnectionConfig(ConnectorType type) {
return frameworkConnectionConfigs.get(type);
}
public void setFrameworkConnectionConfig(ConnectorType type, Object config) {
frameworkConnectionConfigs.put(type, config);
}
public Object getConfigFrameworkJob() {
@ -219,22 +230,14 @@ public class SubmissionRequest {
configFrameworkJob = config;
}
public MutableMapContext getConnectorContext() {
return connectorContext;
public MutableMapContext getConnectorContext(ConnectorType type) {
return connectorContexts.get(type);
}
public MutableMapContext getFrameworkContext() {
return frameworkContext;
}
public String getOutputDirectory() {
return outputDirectory;
}
public void setOutputDirectory(String outputDirectory) {
this.outputDirectory = outputDirectory;
}
public String getNotificationUrl() {
return notificationUrl;
}

Просмотреть файл

@ -0,0 +1,31 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.sqoop.framework.configuration;
import org.apache.sqoop.model.ConfigurationClass;
import org.apache.sqoop.model.Form;
@ConfigurationClass
public class JobConfiguration {
@Form public ThrottlingForm throttling;
public JobConfiguration() {
throttling = new ThrottlingForm();
}
}

Просмотреть файл

@ -18,6 +18,7 @@
package org.apache.sqoop.repository;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.ConnectorManager;
import org.apache.sqoop.connector.spi.MetadataUpgrader;
@ -37,7 +38,6 @@ import org.apache.sqoop.utils.ClassUtils;
import org.apache.sqoop.validation.Validation;
import org.apache.sqoop.validation.Validator;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
@ -446,16 +446,18 @@ public abstract class Repository {
// Make a new copy of the forms from the connector,
// else the values will get set in the forms in the connector for
// each connection.
List<MForm> forms = newConnector.getJobForms(job.getType()).clone(false).getForms();
MJobForms newJobForms = new MJobForms(job.getType(), forms);
upgrader.upgrade(job.getConnectorPart(), newJobForms);
MJob newJob = new MJob(job, newJobForms, job.getFrameworkPart());
List<MForm> forms = newConnector.getJobForms(ConnectorType.FROM).clone(false).getForms();
MJobForms newJobForms = new MJobForms(forms);
upgrader.upgrade(job.getConnectorPart(ConnectorType.FROM), newJobForms);
// @TODO(Abe): Check From and To
MJob newJob = new MJob(job, newJobForms, job.getFrameworkPart(), newJobForms);
// Transform form structures to objects for validations
Object newConfigurationObject = ClassUtils.instantiate(connector.getJobConfigurationClass(job.getType()));
FormUtils.fromForms(newJob.getConnectorPart().getForms(), newConfigurationObject);
// @TODO(Abe): Check From and To
Object newConfigurationObject = ClassUtils.instantiate(connector.getJobConfigurationClass(ConnectorType.FROM));
FormUtils.fromForms(newJob.getConnectorPart(ConnectorType.FROM).getForms(), newConfigurationObject);
Validation validation = validator.validateJob(newJob.getType(), newConfigurationObject);
Validation validation = validator.validateJob(newConfigurationObject);
if (validation.getStatus().canProceed()) {
updateJob(newJob, tx);
} else {
@ -509,6 +511,7 @@ public abstract class Repository {
// Make a new copy of the forms from the connector,
// else the values will get set in the forms in the connector for
// each connection.
// @TODO(Abe): From/To connection forms.
List<MForm> forms = framework.getConnectionForms().clone(false).getForms();
MConnectionForms newConnectionForms = new MConnectionForms(forms);
upgrader.upgrade(connection.getFrameworkPart(), newConnectionForms);
@ -530,16 +533,16 @@ public abstract class Repository {
// Make a new copy of the forms from the framework,
// else the values will get set in the forms in the connector for
// each connection.
List<MForm> forms = framework.getJobForms(job.getType()).clone(false).getForms();
MJobForms newJobForms = new MJobForms(job.getType(), forms);
List<MForm> forms = framework.getJobForms().clone(false).getForms();
MJobForms newJobForms = new MJobForms(forms);
upgrader.upgrade(job.getFrameworkPart(), newJobForms);
MJob newJob = new MJob(job, job.getConnectorPart(), newJobForms);
MJob newJob = new MJob(job, job.getConnectorPart(ConnectorType.FROM), newJobForms, job.getConnectorPart(ConnectorType.TO));
// Transform form structures to objects for validations
Object newConfigurationObject = ClassUtils.instantiate(FrameworkManager.getInstance().getJobConfigurationClass(job.getType()));
Object newConfigurationObject = ClassUtils.instantiate(FrameworkManager.getInstance().getJobConfigurationClass());
FormUtils.fromForms(newJob.getFrameworkPart().getForms(), newConfigurationObject);
Validation validation = validator.validateJob(newJob.getType(), newConfigurationObject);
Validation validation = validator.validateJob(newConfigurationObject);
if (validation.getStatus().canProceed()) {
updateJob(newJob, tx);
} else {

Просмотреть файл

@ -20,22 +20,14 @@ package org.apache.sqoop.execution.mapreduce;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.io.NullWritable;
import org.apache.sqoop.common.MutableMapContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.framework.ExecutionEngine;
import org.apache.sqoop.framework.SubmissionRequest;
import org.apache.sqoop.framework.configuration.ExportJobConfiguration;
import org.apache.sqoop.framework.configuration.ImportJobConfiguration;
import org.apache.sqoop.framework.configuration.OutputFormat;
import org.apache.sqoop.framework.configuration.JobConfiguration;
import org.apache.sqoop.job.JobConstants;
import org.apache.sqoop.job.MapreduceExecutionError;
import org.apache.sqoop.job.etl.Exporter;
import org.apache.sqoop.job.etl.HdfsExportExtractor;
import org.apache.sqoop.job.etl.HdfsExportPartitioner;
import org.apache.sqoop.job.etl.HdfsSequenceImportLoader;
import org.apache.sqoop.job.etl.HdfsTextImportLoader;
import org.apache.sqoop.job.etl.Importer;
import org.apache.sqoop.job.etl.From;
import org.apache.sqoop.job.etl.To;
import org.apache.sqoop.job.io.Data;
import org.apache.sqoop.job.io.SqoopWritable;
import org.apache.sqoop.job.mr.SqoopFileOutputFormat;
import org.apache.sqoop.job.mr.SqoopInputFormat;
import org.apache.sqoop.job.mr.SqoopMapper;
import org.apache.sqoop.job.mr.SqoopNullOutputFormat;
@ -69,99 +61,66 @@ public class MapreduceExecutionEngine extends ExecutionEngine {
request.setOutputValueClass(NullWritable.class);
// Set up framework context
From from = (From)request.getFromCallback();
To to = (To)request.getToCallback();
MutableMapContext context = request.getFrameworkContext();
context.setString(JobConstants.JOB_ETL_PARTITIONER, from.getPartitioner().getName());
context.setString(JobConstants.JOB_ETL_EXTRACTOR, from.getExtractor().getName());
context.setString(JobConstants.JOB_ETL_LOADER, to.getLoader().getName());
context.setString(JobConstants.JOB_ETL_DESTROYER, from.getDestroyer().getName());
context.setString(JobConstants.INTERMEDIATE_DATA_FORMAT,
request.getIntermediateDataFormat().getName());
request.getIntermediateDataFormat().getName());
if(request.getExtractors() != null) {
context.setInteger(JobConstants.JOB_ETL_EXTRACTOR_NUM, request.getExtractors());
}
}
/**
* {@inheritDoc}
*/
@Override
public void prepareImportSubmission(SubmissionRequest gRequest) {
MRSubmissionRequest request = (MRSubmissionRequest) gRequest;
prepareSubmission(request);
request.setOutputFormatClass(SqoopFileOutputFormat.class);
ImportJobConfiguration jobConf = (ImportJobConfiguration) request.getConfigFrameworkJob();
Importer importer = (Importer)request.getConnectorCallbacks();
// Set up framework context
MutableMapContext context = request.getFrameworkContext();
context.setString(JobConstants.JOB_ETL_PARTITIONER, importer.getPartitioner().getName());
context.setString(JobConstants.JOB_ETL_EXTRACTOR, importer.getExtractor().getName());
context.setString(JobConstants.JOB_ETL_DESTROYER, importer.getDestroyer().getName());
// TODO: This settings should be abstracted to core module at some point
if(jobConf.output.outputFormat == OutputFormat.TEXT_FILE) {
context.setString(JobConstants.JOB_ETL_LOADER, HdfsTextImportLoader.class.getName());
} else if(jobConf.output.outputFormat == OutputFormat.SEQUENCE_FILE) {
context.setString(JobConstants.JOB_ETL_LOADER, HdfsSequenceImportLoader.class.getName());
} else {
throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0024,
"Format: " + jobConf.output.outputFormat);
}
if(getCompressionCodecName(jobConf) != null) {
context.setString(JobConstants.HADOOP_COMPRESS_CODEC,
getCompressionCodecName(jobConf));
context.setBoolean(JobConstants.HADOOP_COMPRESS, true);
if(request.getExtractors() != null) {
context.setInteger(JobConstants.JOB_ETL_EXTRACTOR_NUM, request.getExtractors());
}
// @TODO(Abe): Move to HDFS connector.
// if(jobConf.output.outputFormat == OutputFormat.TEXT_FILE) {
// context.setString(JobConstants.JOB_ETL_LOADER, HdfsTextImportLoader.class.getName());
// } else if(jobConf.output.outputFormat == OutputFormat.SEQUENCE_FILE) {
// context.setString(JobConstants.JOB_ETL_LOADER, HdfsSequenceImportLoader.class.getName());
// } else {
// throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0024,
// "Format: " + jobConf.output.outputFormat);
// }
// if(getCompressionCodecName(jobConf) != null) {
// context.setString(JobConstants.HADOOP_COMPRESS_CODEC,
// getCompressionCodecName(jobConf));
// context.setBoolean(JobConstants.HADOOP_COMPRESS, true);
// }
}
private String getCompressionCodecName(ImportJobConfiguration jobConf) {
if(jobConf.output.compression == null)
return null;
switch(jobConf.output.compression) {
case NONE:
return null;
case DEFAULT:
return "org.apache.hadoop.io.compress.DefaultCodec";
case DEFLATE:
return "org.apache.hadoop.io.compress.DeflateCodec";
case GZIP:
return "org.apache.hadoop.io.compress.GzipCodec";
case BZIP2:
return "org.apache.hadoop.io.compress.BZip2Codec";
case LZO:
return "com.hadoop.compression.lzo.LzoCodec";
case LZ4:
return "org.apache.hadoop.io.compress.Lz4Codec";
case SNAPPY:
return "org.apache.hadoop.io.compress.SnappyCodec";
case CUSTOM:
return jobConf.output.customCompression.trim();
}
return null;
}
/**
* {@inheritDoc}
*/
@Override
public void prepareExportSubmission(SubmissionRequest gRequest) {
MRSubmissionRequest request = (MRSubmissionRequest) gRequest;
ExportJobConfiguration jobConf = (ExportJobConfiguration) request.getConfigFrameworkJob();
prepareSubmission(request);
Exporter exporter = (Exporter)request.getConnectorCallbacks();
// Set up framework context
MutableMapContext context = request.getFrameworkContext();
context.setString(JobConstants.JOB_ETL_PARTITIONER, HdfsExportPartitioner.class.getName());
context.setString(JobConstants.JOB_ETL_LOADER, exporter.getLoader().getName());
context.setString(JobConstants.JOB_ETL_DESTROYER, exporter.getDestroyer().getName());
// Extractor that will be able to read all supported file types
context.setString(JobConstants.JOB_ETL_EXTRACTOR, HdfsExportExtractor.class.getName());
context.setString(JobConstants.HADOOP_INPUTDIR, jobConf.input.inputDirectory);
}
// @TODO(Abe): Move to HDFS connector.
// private String getCompressionCodecName(ImportJobConfiguration jobConf) {
// if(jobConf.output.compression == null)
// return null;
// switch(jobConf.output.compression) {
// case NONE:
// return null;
// case DEFAULT:
// return "org.apache.hadoop.io.compress.DefaultCodec";
// case DEFLATE:
// return "org.apache.hadoop.io.compress.DeflateCodec";
// case GZIP:
// return "org.apache.hadoop.io.compress.GzipCodec";
// case BZIP2:
// return "org.apache.hadoop.io.compress.BZip2Codec";
// case LZO:
// return "com.hadoop.compression.lzo.LzoCodec";
// case LZ4:
// return "org.apache.hadoop.io.compress.Lz4Codec";
// case SNAPPY:
// return "org.apache.hadoop.io.compress.SnappyCodec";
// case CUSTOM:
// return jobConf.output.customCompression.trim();
// }
// return null;
// }
/**
* Our execution engine have additional dependencies that needs to be available

Просмотреть файл

@ -51,8 +51,11 @@ public final class JobConstants extends Constants {
public static final String JOB_ETL_EXTRACTOR_NUM = PREFIX_JOB_CONFIG
+ "etl.extractor.count";
public static final String PREFIX_CONNECTOR_CONTEXT =
PREFIX_JOB_CONFIG + "connector.context.";
public static final String PREFIX_CONNECTOR_FROM_CONTEXT =
PREFIX_JOB_CONFIG + "connector.from.context.";
public static final String PREFIX_CONNECTOR_TO_CONTEXT =
PREFIX_JOB_CONFIG + "connector.to.context.";
// Hadoop specific constants
// We're using constants from Hadoop 1. Hadoop 2 has different names, but

Просмотреть файл

@ -41,154 +41,154 @@ import org.apache.sqoop.job.PrefixContext;
* Extract from HDFS.
* Default field delimiter of a record is comma.
*/
public class HdfsExportExtractor extends Extractor<ConnectionConfiguration, ExportJobConfiguration, HdfsExportPartition> {
public static final Logger LOG = Logger.getLogger(HdfsExportExtractor.class);
private Configuration conf;
private DataWriter dataWriter;
private long rowRead = 0;
@Override
public void extract(ExtractorContext context,
ConnectionConfiguration connectionConfiguration,
ExportJobConfiguration jobConfiguration, HdfsExportPartition partition) {
conf = ((PrefixContext) context.getContext()).getConfiguration();
dataWriter = context.getDataWriter();
try {
HdfsExportPartition p = partition;
LOG.info("Working on partition: " + p);
int numFiles = p.getNumberOfFiles();
for (int i = 0; i < numFiles; i++) {
extractFile(p.getFile(i), p.getOffset(i), p.getLength(i));
}
} catch (IOException e) {
throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0017, e);
}
}
private void extractFile(Path file, long start, long length)
throws IOException {
long end = start + length;
LOG.info("Extracting file " + file);
LOG.info("\t from offset " + start);
LOG.info("\t to offset " + end);
LOG.info("\t of length " + length);
if(isSequenceFile(file)) {
extractSequenceFile(file, start, length);
} else {
extractTextFile(file, start, length);
}
}
/**
* Extracts Sequence file
* @param file
* @param start
* @param length
* @throws IOException
*/
private void extractSequenceFile(Path file, long start, long length)
throws IOException {
LOG.info("Extracting sequence file");
long end = start + length;
SequenceFile.Reader filereader = new SequenceFile.Reader(
file.getFileSystem(conf), file, conf);
if (start > filereader.getPosition()) {
filereader.sync(start); // sync to start
}
Text line = new Text();
boolean hasNext = filereader.next(line);
while (hasNext) {
rowRead++;
dataWriter.writeStringRecord(line.toString());
line = new Text();
hasNext = filereader.next(line);
if (filereader.getPosition() >= end && filereader.syncSeen()) {
break;
}
}
filereader.close();
}
/**
* Extracts Text file
* @param file
* @param start
* @param length
* @throws IOException
*/
private void extractTextFile(Path file, long start, long length)
throws IOException {
LOG.info("Extracting text file");
long end = start + length;
FileSystem fs = file.getFileSystem(conf);
FSDataInputStream filestream = fs.open(file);
CompressionCodec codec = (new CompressionCodecFactory(conf)).getCodec(file);
LineReader filereader;
Seekable fileseeker = filestream;
// Hadoop 1.0 does not have support for custom record delimiter and thus
// we
// are supporting only default one.
// We might add another "else if" case for SplittableCompressionCodec once
// we drop support for Hadoop 1.0.
if (codec == null) {
filestream.seek(start);
filereader = new LineReader(filestream);
} else {
filereader = new LineReader(codec.createInputStream(filestream,
codec.createDecompressor()), conf);
fileseeker = filestream;
}
if (start != 0) {
// always throw away first record because
// one extra line is read in previous split
start += filereader.readLine(new Text(), 0);
}
int size;
LOG.info("Start position: " + String.valueOf(start));
long next = start;
while (next <= end) {
Text line = new Text();
size = filereader.readLine(line, Integer.MAX_VALUE);
if (size == 0) {
break;
}
if (codec == null) {
next += size;
} else {
next = fileseeker.getPos();
}
rowRead++;
dataWriter.writeStringRecord(line.toString());
}
LOG.info("Extracting ended on position: " + fileseeker.getPos());
filestream.close();
}
@Override
public long getRowsRead() {
return rowRead;
}
/**
* Returns true if given file is sequence
* @param file
* @return boolean
*/
private boolean isSequenceFile(Path file) {
SequenceFile.Reader filereader = null;
try {
filereader = new SequenceFile.Reader(file.getFileSystem(conf), file, conf);
filereader.close();
} catch (IOException e) {
return false;
}
return true;
}
}
//public class HdfsExportExtractor extends Extractor<ConnectionConfiguration, ExportJobConfiguration, HdfsExportPartition> {
//
// public static final Logger LOG = Logger.getLogger(HdfsExportExtractor.class);
//
// private Configuration conf;
// private DataWriter dataWriter;
// private long rowRead = 0;
//
// @Override
// public void extract(ExtractorContext context,
// ConnectionConfiguration connectionConfiguration,
// ExportJobConfiguration jobConfiguration, HdfsExportPartition partition) {
//
// conf = ((PrefixContext) context.getContext()).getConfiguration();
// dataWriter = context.getDataWriter();
//
// try {
// HdfsExportPartition p = partition;
// LOG.info("Working on partition: " + p);
// int numFiles = p.getNumberOfFiles();
// for (int i = 0; i < numFiles; i++) {
// extractFile(p.getFile(i), p.getOffset(i), p.getLength(i));
// }
// } catch (IOException e) {
// throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0017, e);
// }
// }
//
// private void extractFile(Path file, long start, long length)
// throws IOException {
// long end = start + length;
// LOG.info("Extracting file " + file);
// LOG.info("\t from offset " + start);
// LOG.info("\t to offset " + end);
// LOG.info("\t of length " + length);
// if(isSequenceFile(file)) {
// extractSequenceFile(file, start, length);
// } else {
// extractTextFile(file, start, length);
// }
// }
//
// /**
// * Extracts Sequence file
// * @param file
// * @param start
// * @param length
// * @throws IOException
// */
// private void extractSequenceFile(Path file, long start, long length)
// throws IOException {
// LOG.info("Extracting sequence file");
// long end = start + length;
// SequenceFile.Reader filereader = new SequenceFile.Reader(
// file.getFileSystem(conf), file, conf);
//
// if (start > filereader.getPosition()) {
// filereader.sync(start); // sync to start
// }
//
// Text line = new Text();
// boolean hasNext = filereader.next(line);
// while (hasNext) {
// rowRead++;
// dataWriter.writeStringRecord(line.toString());
// line = new Text();
// hasNext = filereader.next(line);
// if (filereader.getPosition() >= end && filereader.syncSeen()) {
// break;
// }
// }
// filereader.close();
// }
//
// /**
// * Extracts Text file
// * @param file
// * @param start
// * @param length
// * @throws IOException
// */
// private void extractTextFile(Path file, long start, long length)
// throws IOException {
// LOG.info("Extracting text file");
// long end = start + length;
// FileSystem fs = file.getFileSystem(conf);
// FSDataInputStream filestream = fs.open(file);
// CompressionCodec codec = (new CompressionCodecFactory(conf)).getCodec(file);
// LineReader filereader;
// Seekable fileseeker = filestream;
//
// // Hadoop 1.0 does not have support for custom record delimiter and thus
// // we
// // are supporting only default one.
// // We might add another "else if" case for SplittableCompressionCodec once
// // we drop support for Hadoop 1.0.
// if (codec == null) {
// filestream.seek(start);
// filereader = new LineReader(filestream);
// } else {
// filereader = new LineReader(codec.createInputStream(filestream,
// codec.createDecompressor()), conf);
// fileseeker = filestream;
// }
// if (start != 0) {
// // always throw away first record because
// // one extra line is read in previous split
// start += filereader.readLine(new Text(), 0);
// }
// int size;
// LOG.info("Start position: " + String.valueOf(start));
// long next = start;
// while (next <= end) {
// Text line = new Text();
// size = filereader.readLine(line, Integer.MAX_VALUE);
// if (size == 0) {
// break;
// }
// if (codec == null) {
// next += size;
// } else {
// next = fileseeker.getPos();
// }
// rowRead++;
// dataWriter.writeStringRecord(line.toString());
// }
// LOG.info("Extracting ended on position: " + fileseeker.getPos());
// filestream.close();
// }
//
// @Override
// public long getRowsRead() {
// return rowRead;
// }
//
// /**
// * Returns true if given file is sequence
// * @param file
// * @return boolean
// */
// private boolean isSequenceFile(Path file) {
// SequenceFile.Reader filereader = null;
// try {
// filereader = new SequenceFile.Reader(file.getFileSystem(conf), file, conf);
// filereader.close();
// } catch (IOException e) {
// return false;
// }
// return true;
// }
//}

Просмотреть файл

@ -22,10 +22,10 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.log4j.PropertyConfigurator;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.job.JobConstants;
import org.apache.sqoop.json.util.SchemaSerialization;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.utils.ClassUtils;
import org.json.simple.JSONObject;
@ -40,80 +40,98 @@ import java.util.Properties;
*/
public final class ConfigurationUtils {
private static final String JOB_TYPE = JobConstants.PREFIX_JOB_CONFIG + "type";
private static final String JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.class.connector.from.connection";
private static final String JOB_CONFIG_CLASS_CONNECTOR_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.class.connector.connection";
private static final String JOB_CONFIG_CLASS_TO_CONNECTOR_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.class.connector.to.connection";
private static final String JOB_CONFIG_CLASS_CONNECTOR_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.class.connector.job";
private static final String JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.class.connector.from.job";
private static final String JOB_CONFIG_CLASS_FRAMEWORK_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.class.framework.connection";
private static final String JOB_CONFIG_CLASS_TO_CONNECTOR_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.class.connector.to.job";
private static final String JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.class.framework.from.connection";
private static final String JOB_CONFIG_CLASS_TO_FRAMEWORK_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.class.framework.to.connection";
private static final String JOB_CONFIG_CLASS_FRAMEWORK_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.class.framework.job";
private static final String JOB_CONFIG_CONNECTOR_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.connector.connection";
private static final String JOB_CONFIG_FROM_CONNECTOR_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.connector.from.connection";
private static final Text JOB_CONFIG_CONNECTOR_CONNECTION_KEY = new Text(JOB_CONFIG_CONNECTOR_CONNECTION);
private static final Text JOB_CONFIG_FROM_CONNECTOR_CONNECTION_KEY = new Text(JOB_CONFIG_FROM_CONNECTOR_CONNECTION);
private static final String JOB_CONFIG_CONNECTOR_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.connector.job";
private static final String JOB_CONFIG_TO_CONNECTOR_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.connector.to.connection";
private static final Text JOB_CONFIG_CONNECTOR_JOB_KEY = new Text(JOB_CONFIG_CONNECTOR_JOB);
private static final Text JOB_CONFIG_TO_CONNECTOR_CONNECTION_KEY = new Text(JOB_CONFIG_TO_CONNECTOR_CONNECTION);
private static final String JOB_CONFIG_FRAMEWORK_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.framework.connection";
private static final String JOB_CONFIG_FROM_CONNECTOR_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.connector.from.job";
private static final Text JOB_CONFIG_FRAMEWORK_CONNECTION_KEY = new Text(JOB_CONFIG_FRAMEWORK_CONNECTION);
private static final Text JOB_CONFIG_FROM_CONNECTOR_JOB_KEY = new Text(JOB_CONFIG_FROM_CONNECTOR_JOB);
private static final String JOB_CONFIG_TO_CONNECTOR_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.connector.to.job";
private static final Text JOB_CONFIG_TO_CONNECTOR_JOB_KEY = new Text(JOB_CONFIG_TO_CONNECTOR_JOB);
private static final String JOB_CONFIG_FROM_FRAMEWORK_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.framework.from.connection";
private static final Text JOB_CONFIG_FROM_FRAMEWORK_CONNECTION_KEY = new Text(JOB_CONFIG_FROM_FRAMEWORK_CONNECTION);
private static final String JOB_CONFIG_TO_FRAMEWORK_CONNECTION = JobConstants.PREFIX_JOB_CONFIG + "config.framework.from.connection";
private static final Text JOB_CONFIG_TO_FRAMEWORK_CONNECTION_KEY = new Text(JOB_CONFIG_TO_FRAMEWORK_CONNECTION);
private static final String JOB_CONFIG_FRAMEWORK_JOB = JobConstants.PREFIX_JOB_CONFIG + "config.framework.job";
private static final Text JOB_CONFIG_FRAMEWORK_JOB_KEY = new Text(JOB_CONFIG_FRAMEWORK_JOB);
private static final String SCHEMA_CONNECTOR = JobConstants.PREFIX_JOB_CONFIG + "schema.connector";
private static final String SCHEMA_FROM_CONNECTOR = JobConstants.PREFIX_JOB_CONFIG + "schema.connector.from";
private static final Text SCHEMA_CONNECTOR_KEY = new Text(SCHEMA_CONNECTOR);
private static final Text SCHEMA_FROM_CONNECTOR_KEY = new Text(SCHEMA_FROM_CONNECTOR);
private static final String SCHEMA_TO_CONNECTOR = JobConstants.PREFIX_JOB_CONFIG + "schema.connector.to";
private static final Text SCHEMA_TO_CONNECTOR_KEY = new Text(SCHEMA_TO_CONNECTOR);
private static final String SCHEMA_HIO = JobConstants.PREFIX_JOB_CONFIG + "schema.hio";
private static final Text SCHEMA_HIO_KEY = new Text(SCHEMA_HIO);
/**
* Persist job type in the configuration object.
*
* @param configuration MapReduce configuration object
* @param type Job type
*/
public static void setJobType(Configuration configuration, MJob.Type type) {
configuration.set(JOB_TYPE, type.name());
}
/**
* Retrieve job type.
*
* @param configuration MapReduce configuration object
* @return Job type
*/
public static MJob.Type getJobType(Configuration configuration) {
return MJob.Type.valueOf(configuration.get(JOB_TYPE));
}
/**
* Persist Connector configuration object for connection.
*
* @param job MapReduce job object
* @param obj Configuration object
*/
public static void setConfigConnectorConnection(Job job, Object obj) {
job.getConfiguration().set(JOB_CONFIG_CLASS_CONNECTOR_CONNECTION, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_CONNECTOR_CONNECTION_KEY, FormUtils.toJson(obj).getBytes());
public static void setConnectorConnectionConfig(ConnectorType type, Job job, Object obj) {
switch (type) {
case FROM:
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_FROM_CONNECTOR_CONNECTION_KEY, FormUtils.toJson(obj).getBytes());
break;
case TO:
job.getConfiguration().set(JOB_CONFIG_CLASS_TO_CONNECTOR_CONNECTION, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_TO_CONNECTOR_CONNECTION_KEY, FormUtils.toJson(obj).getBytes());
break;
}
}
/**
* Persist Connector configuration object for job.
* Persist Connector configuration objects for job.
*
* @param job MapReduce job object
* @param obj Configuration object
*/
public static void setConfigConnectorJob(Job job, Object obj) {
job.getConfiguration().set(JOB_CONFIG_CLASS_CONNECTOR_JOB, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_CONNECTOR_JOB_KEY, FormUtils.toJson(obj).getBytes());
public static void setConnectorJobConfig(ConnectorType type, Job job, Object obj) {
switch (type) {
case FROM:
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_FROM_CONNECTOR_JOB_KEY, FormUtils.toJson(obj).getBytes());
break;
case TO:
job.getConfiguration().set(JOB_CONFIG_CLASS_TO_CONNECTOR_JOB, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_TO_CONNECTOR_JOB_KEY, FormUtils.toJson(obj).getBytes());
break;
}
}
/**
@ -122,9 +140,18 @@ public final class ConfigurationUtils {
* @param job MapReduce job object
* @param obj Configuration object
*/
public static void setConfigFrameworkConnection(Job job, Object obj) {
job.getConfiguration().set(JOB_CONFIG_CLASS_FRAMEWORK_CONNECTION, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_FRAMEWORK_CONNECTION_KEY, FormUtils.toJson(obj).getBytes());
public static void setFrameworkConnectionConfig(ConnectorType type, Job job, Object obj) {
switch (type) {
case FROM:
job.getConfiguration().set(JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_FROM_FRAMEWORK_CONNECTION_KEY, FormUtils.toJson(obj).getBytes());
break;
case TO:
job.getConfiguration().set(JOB_CONFIG_CLASS_TO_FRAMEWORK_CONNECTION, obj.getClass().getName());
job.getCredentials().addSecretKey(JOB_CONFIG_TO_FRAMEWORK_CONNECTION_KEY, FormUtils.toJson(obj).getBytes());
break;
}
}
/**
@ -144,8 +171,16 @@ public final class ConfigurationUtils {
* @param configuration MapReduce configuration object
* @return Configuration object
*/
public static Object getConfigConnectorConnection(Configuration configuration) {
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_CONNECTOR_CONNECTION, JOB_CONFIG_CONNECTOR_CONNECTION_KEY);
public static Object getConnectorConnectionConfig(ConnectorType type, Configuration configuration) {
switch (type) {
case FROM:
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_CONNECTOR_CONNECTION, JOB_CONFIG_FROM_CONNECTOR_CONNECTION_KEY);
case TO:
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_TO_CONNECTOR_CONNECTION, JOB_CONFIG_TO_CONNECTOR_CONNECTION_KEY);
}
return null;
}
/**
@ -154,8 +189,16 @@ public final class ConfigurationUtils {
* @param configuration MapReduce configuration object
* @return Configuration object
*/
public static Object getConfigConnectorJob(Configuration configuration) {
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_CONNECTOR_JOB, JOB_CONFIG_CONNECTOR_JOB_KEY);
public static Object getConnectorJobConfig(ConnectorType type, Configuration configuration) {
switch (type) {
case FROM:
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_CONNECTOR_JOB, JOB_CONFIG_FROM_CONNECTOR_JOB_KEY);
case TO:
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_TO_CONNECTOR_JOB, JOB_CONFIG_TO_CONNECTOR_JOB_KEY);
}
return null;
}
/**
@ -164,8 +207,16 @@ public final class ConfigurationUtils {
* @param configuration MapReduce configuration object
* @return Configuration object
*/
public static Object getConfigFrameworkConnection(Configuration configuration) {
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FRAMEWORK_CONNECTION, JOB_CONFIG_FRAMEWORK_CONNECTION_KEY);
public static Object getFrameworkConnectionConfig(ConnectorType type, Configuration configuration) {
switch (type) {
case FROM:
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_FROM_FRAMEWORK_CONNECTION, JOB_CONFIG_FROM_FRAMEWORK_CONNECTION_KEY);
case TO:
return loadConfiguration((JobConf) configuration, JOB_CONFIG_CLASS_TO_FRAMEWORK_CONNECTION, JOB_CONFIG_TO_FRAMEWORK_CONNECTION_KEY);
}
return null;
}
/**
@ -179,14 +230,26 @@ public final class ConfigurationUtils {
}
/**
* Persist Connector generated schema.
* Persist From Connector generated schema.
*
* @param job MapReduce Job object
* @param schema Schema
*/
public static void setConnectorSchema(Job job, Schema schema) {
public static void setFromConnectorSchema(Job job, Schema schema) {
if(schema != null) {
job.getCredentials().addSecretKey(SCHEMA_CONNECTOR_KEY, SchemaSerialization.extractSchema(schema).toJSONString().getBytes());
job.getCredentials().addSecretKey(SCHEMA_FROM_CONNECTOR_KEY, SchemaSerialization.extractSchema(schema).toJSONString().getBytes());
}
}
/**
* Persist To Connector generated schema.
*
* @param job MapReduce Job object
* @param schema Schema
*/
public static void setToConnectorSchema(Job job, Schema schema) {
if(schema != null) {
job.getCredentials().addSecretKey(SCHEMA_TO_CONNECTOR_KEY, SchemaSerialization.extractSchema(schema).toJSONString().getBytes());
}
}
@ -203,23 +266,21 @@ public final class ConfigurationUtils {
}
/**
* Retrieve Connector generated schema.
* Retrieve From Connector generated schema.
*
* @param configuration MapReduce configuration object
* @return Schema
*/
public static Schema getConnectorSchema(Configuration configuration) {
return getSchemaFromBytes(((JobConf) configuration).getCredentials().getSecretKey(SCHEMA_CONNECTOR_KEY));
}
public static Schema getConnectorSchema(ConnectorType type, Configuration configuration) {
switch (type) {
case FROM:
return getSchemaFromBytes(((JobConf) configuration).getCredentials().getSecretKey(SCHEMA_FROM_CONNECTOR_KEY));
/**
* Retrieve Framework generated schema.
*
* @param configuration MapReduce configuration object
* @return Schema
*/
public static Schema getHioSchema(Configuration configuration) {
return getSchemaFromBytes(((JobConf) configuration).getCredentials().getSecretKey(SCHEMA_HIO_KEY));
case TO:
return getSchemaFromBytes(((JobConf) configuration).getCredentials().getSecretKey(SCHEMA_TO_CONNECTOR_KEY));
}
return null;
}
/**

Просмотреть файл

@ -19,10 +19,12 @@ package org.apache.sqoop.job.mr;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.job.JobConstants;
import org.apache.sqoop.job.PrefixContext;
import org.apache.sqoop.job.etl.Destroyer;
import org.apache.sqoop.job.etl.DestroyerContext;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.utils.ClassUtils;
@ -51,18 +53,18 @@ public class SqoopDestroyerExecutor {
}
// Objects that should be pass to the Destroyer execution
PrefixContext subContext = new PrefixContext(configuration, JobConstants.PREFIX_CONNECTOR_CONTEXT);
Object configConnection = ConfigurationUtils.getConfigConnectorConnection(configuration);
Object configJob = ConfigurationUtils.getConfigConnectorJob(configuration);
PrefixContext subContext = new PrefixContext(configuration, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
Object fromConfigConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.FROM, configuration);
Object fromConfigJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.FROM, configuration);
// Propagate connector schema in every case for now
// TODO: Change to coditional choosing between HIO and Connector schema
Schema schema = ConfigurationUtils.getConnectorSchema(configuration);
// TODO: Change to coditional choosing between Connector schemas.
Schema schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, configuration);
DestroyerContext destroyerContext = new DestroyerContext(subContext, success, schema);
LOG.info("Executing destroyer class " + destroyer.getClass());
destroyer.destroy(destroyerContext, configConnection, configJob);
destroyer.destroy(destroyerContext, fromConfigConnection, fromConfigJob);
}
private SqoopDestroyerExecutor() {

Просмотреть файл

@ -29,6 +29,7 @@ import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.job.JobConstants;
import org.apache.sqoop.job.MapreduceExecutionError;
@ -36,6 +37,7 @@ import org.apache.sqoop.job.PrefixContext;
import org.apache.sqoop.job.etl.Partition;
import org.apache.sqoop.job.etl.Partitioner;
import org.apache.sqoop.job.etl.PartitionerContext;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.utils.ClassUtils;
@ -61,10 +63,10 @@ public class SqoopInputFormat extends InputFormat<SqoopSplit, NullWritable> {
String partitionerName = conf.get(JobConstants.JOB_ETL_PARTITIONER);
Partitioner partitioner = (Partitioner) ClassUtils.instantiate(partitionerName);
PrefixContext connectorContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_CONTEXT);
Object connectorConnection = ConfigurationUtils.getConfigConnectorConnection(conf);
Object connectorJob = ConfigurationUtils.getConfigConnectorJob(conf);
Schema schema = ConfigurationUtils.getConnectorSchema(conf);
PrefixContext connectorContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
Object connectorConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.FROM, conf);
Object connectorJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.FROM, conf);
Schema schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, conf);
long maxPartitions = conf.getLong(JobConstants.JOB_ETL_EXTRACTOR_NUM, 10);
PartitionerContext partitionerContext = new PartitionerContext(connectorContext, maxPartitions, schema);

Просмотреть файл

@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
import org.apache.sqoop.job.JobConstants;
@ -34,6 +35,7 @@ import org.apache.sqoop.job.PrefixContext;
import org.apache.sqoop.job.etl.Extractor;
import org.apache.sqoop.job.etl.ExtractorContext;
import org.apache.sqoop.etl.io.DataWriter;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.job.io.SqoopWritable;
import org.apache.sqoop.submission.counter.SqoopCounters;
@ -75,24 +77,13 @@ public class SqoopMapper extends Mapper<SqoopSplit, NullWritable, SqoopWritable,
Object configJob = null;
// Propagate connector schema in every case for now
// TODO: Change to coditional choosing between HIO and Connector schema
Schema schema = ConfigurationUtils.getConnectorSchema(conf);
// TODO: Change to coditional choosing between Connector schemas.
Schema schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, conf);
// Executor is in connector space for IMPORT and in framework space for EXPORT
switch (ConfigurationUtils.getJobType(conf)) {
case IMPORT:
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_CONTEXT);
configConnection = ConfigurationUtils.getConfigConnectorConnection(conf);
configJob = ConfigurationUtils.getConfigConnectorJob(conf);
break;
case EXPORT:
subContext = new PrefixContext(conf, "");
configConnection = ConfigurationUtils.getConfigFrameworkConnection(conf);
configJob = ConfigurationUtils.getConfigFrameworkJob(conf);
break;
default:
throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0023);
}
// Get configs for extractor
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT);
configConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.FROM, conf);
configJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.FROM, conf);
SqoopSplit split = context.getCurrentKey();
ExtractorContext extractorContext = new ExtractorContext(subContext, new MapDataWriter(context), schema);

Просмотреть файл

@ -30,6 +30,7 @@ import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
@ -39,6 +40,7 @@ import org.apache.sqoop.job.PrefixContext;
import org.apache.sqoop.job.etl.Loader;
import org.apache.sqoop.job.etl.LoaderContext;
import org.apache.sqoop.etl.io.DataReader;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.schema.Schema;
import org.apache.sqoop.job.io.SqoopWritable;
import org.apache.sqoop.utils.ClassUtils;
@ -225,23 +227,13 @@ public class SqoopOutputFormatLoadExecutor {
if (!isTest) {
// Propagate connector schema in every case for now
// TODO: Change to coditional choosing between HIO and Connector schema
schema = ConfigurationUtils.getConnectorSchema(conf);
// TODO: Change to coditional choosing between Connector schemas.
// @TODO(Abe): Maybe use TO schema?
schema = ConfigurationUtils.getConnectorSchema(ConnectorType.FROM, conf);
switch (ConfigurationUtils.getJobType(conf)) {
case EXPORT:
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_CONTEXT);
configConnection = ConfigurationUtils.getConfigConnectorConnection(conf);
configJob = ConfigurationUtils.getConfigConnectorJob(conf);
break;
case IMPORT:
subContext = new PrefixContext(conf, "");
configConnection = ConfigurationUtils.getConfigFrameworkConnection(conf);
configJob = ConfigurationUtils.getConfigFrameworkJob(conf);
break;
default:
throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0023);
}
subContext = new PrefixContext(conf, JobConstants.PREFIX_CONNECTOR_TO_CONTEXT);
configConnection = ConfigurationUtils.getConnectorConnectionConfig(ConnectorType.TO, conf);
configJob = ConfigurationUtils.getConnectorJobConfig(ConnectorType.TO, conf);
}
// Create loader context

Просмотреть файл

@ -36,7 +36,7 @@ import org.apache.hadoop.io.compress.BZip2Codec;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.mapreduce.Job;
//import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
import org.apache.sqoop.job.etl.HdfsExportExtractor;
//import org.apache.sqoop.job.etl.HdfsExportExtractor;
import org.apache.sqoop.job.etl.HdfsExportPartitioner;
import org.apache.sqoop.job.etl.HdfsSequenceImportLoader;
import org.apache.sqoop.job.etl.Loader;

Просмотреть файл

@ -40,8 +40,8 @@ import javax.sql.DataSource;
import org.apache.log4j.Logger;
import org.apache.commons.lang.StringUtils;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.framework.FrameworkManager;
import org.apache.sqoop.model.MBooleanInput;
import org.apache.sqoop.model.MConnection;
import org.apache.sqoop.model.MConnectionForms;
@ -117,11 +117,9 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
registerForms(null, null, mf.getConnectionForms().getForms(),
MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
// Register all jobs
for (MJobForms jobForms : mf.getAllJobsForms().values()) {
registerForms(null, jobForms.getType(), jobForms.getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
}
// Register job forms
registerForms(null, null, mf.getJobForms().getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
} catch (SQLException ex) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0014, mf.toString(), ex);
@ -153,10 +151,10 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
// Register all jobs
for (MJobForms jobForms : mc.getAllJobsForms().values()) {
registerForms(connectorId, jobForms.getType(), jobForms.getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
}
registerForms(connectorId, ConnectorType.FROM, mc.getJobForms(ConnectorType.FROM).getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
registerForms(connectorId, ConnectorType.TO, mc.getJobForms(ConnectorType.TO).getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
} catch (SQLException ex) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0014,
@ -513,10 +511,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
// Register all jobs
for (MJobForms jobForms : mf.getAllJobsForms().values()) {
registerForms(null, jobForms.getType(), jobForms.getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
}
registerForms(null, null, mf.getJobForms().getForms(),
MFormType.JOB.name(), baseFormStmt, baseInputStmt);
// We're using hardcoded value for framework metadata as they are
// represented as NULL in the database.
@ -544,8 +540,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
inputFetchStmt = conn.prepareStatement(STMT_FETCH_INPUT);
List<MForm> connectionForms = new ArrayList<MForm>();
Map<MJob.Type, List<MForm>> jobForms =
new HashMap<MJob.Type, List<MForm>>();
List<MForm> jobForms = new ArrayList<MForm>();
loadForms(connectionForms, jobForms, formFetchStmt, inputFetchStmt, 1);
@ -555,7 +550,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
}
mf = new MFramework(new MConnectionForms(connectionForms),
convertToJobList(jobForms), detectFrameworkVersion(conn));
new MJobForms(jobForms), detectFrameworkVersion(conn));
// We're using hardcoded value for framework metadata as they are
// represented as NULL in the database.
@ -931,8 +926,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
stmt = conn.prepareStatement(STMT_INSERT_JOB,
Statement.RETURN_GENERATED_KEYS);
stmt.setString(1, job.getName());
stmt.setLong(2, job.getConnectionId());
stmt.setString(3, job.getType().name());
stmt.setLong(2, job.getConnectionId(ConnectorType.FROM));
stmt.setLong(3, job.getConnectionId(ConnectorType.TO));
stmt.setBoolean(4, job.getEnabled());
stmt.setString(5, job.getCreationUser());
stmt.setTimestamp(6, new Timestamp(job.getCreationDate().getTime()));
@ -955,12 +950,16 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
createInputValues(STMT_INSERT_JOB_INPUT,
jobId,
job.getConnectorPart().getForms(),
job.getConnectorPart(ConnectorType.FROM).getForms(),
conn);
createInputValues(STMT_INSERT_JOB_INPUT,
jobId,
job.getFrameworkPart().getForms(),
conn);
createInputValues(STMT_INSERT_JOB_INPUT,
jobId,
job.getConnectorPart(ConnectorType.TO).getForms(),
conn);
job.setPersistenceId(jobId);
@ -997,12 +996,12 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
// And reinsert new values
createInputValues(STMT_INSERT_JOB_INPUT,
job.getPersistenceId(),
job.getConnectorPart().getForms(),
job.getConnectorPart(ConnectorType.FROM).getForms(),
conn);
createInputValues(STMT_INSERT_JOB_INPUT,
job.getPersistenceId(),
job.getFrameworkPart().getForms(),
conn);
job.getPersistenceId(),
job.getFrameworkPart().getForms(),
conn);
} catch (SQLException ex) {
logException(ex, job);
@ -1620,14 +1619,14 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
formFetchStmt.setLong(1, connectorId);
List<MForm> connectionForms = new ArrayList<MForm>();
Map<MJob.Type, List<MForm>> jobForms =
new HashMap<MJob.Type, List<MForm>>();
Map<ConnectorType, List<MForm>> jobForms = new HashMap<ConnectorType, List<MForm>>();
loadForms(connectionForms, jobForms, formFetchStmt, inputFetchStmt, 1);
loadConnectorForms(connectionForms, jobForms, formFetchStmt, inputFetchStmt, 1);
MConnector mc = new MConnector(connectorName, connectorClassName, connectorVersion,
new MConnectionForms(connectionForms),
convertToJobList(jobForms));
new MConnectionForms(connectionForms),
new MJobForms(jobForms.get(ConnectorType.FROM)),
new MJobForms(jobForms.get(ConnectorType.TO)));
mc.setPersistenceId(connectorId);
connectors.add(mc);
@ -1674,13 +1673,10 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
List<MForm> connectorConnForms = new ArrayList<MForm>();
List<MForm> frameworkConnForms = new ArrayList<MForm>();
List<MForm> frameworkJobForms = new ArrayList<MForm>();
Map<ConnectorType, List<MForm>> connectorJobForms = new HashMap<ConnectorType, List<MForm>>();
Map<MJob.Type, List<MForm>> connectorJobForms
= new HashMap<MJob.Type, List<MForm>>();
Map<MJob.Type, List<MForm>> frameworkJobForms
= new HashMap<MJob.Type, List<MForm>>();
loadForms(connectorConnForms, connectorJobForms,
loadConnectorForms(connectorConnForms, connectorJobForms,
formConnectorFetchStmt, inputFetchStmt, 2);
loadForms(frameworkConnForms, frameworkJobForms,
formFrameworkFetchStmt, inputFetchStmt, 2);
@ -1725,20 +1721,19 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
inputFetchStmt = conn.prepareStatement(STMT_FETCH_JOB_INPUT);
while(rsJob.next()) {
long connectorId = rsJob.getLong(1);
long id = rsJob.getLong(2);
String name = rsJob.getString(3);
long connectionId = rsJob.getLong(4);
String stringType = rsJob.getString(5);
boolean enabled = rsJob.getBoolean(6);
String createBy = rsJob.getString(7);
Date creationDate = rsJob.getTimestamp(8);
String updateBy = rsJob.getString(9);
Date lastUpdateDate = rsJob.getTimestamp(10);
long fromConnectorId = rsJob.getLong(1);
long toConnectorId = rsJob.getLong(2);
long id = rsJob.getLong(3);
String name = rsJob.getString(4);
long fromConnectionId = rsJob.getLong(5);
long toConnectionId = rsJob.getLong(6);
boolean enabled = rsJob.getBoolean(7);
String createBy = rsJob.getString(8);
Date creationDate = rsJob.getTimestamp(9);
String updateBy = rsJob.getString(10);
Date lastUpdateDate = rsJob.getTimestamp(11);
MJob.Type type = MJob.Type.valueOf(stringType);
formConnectorFetchStmt.setLong(1, connectorId);
formConnectorFetchStmt.setLong(1, fromConnectorId);
inputFetchStmt.setLong(1, id);
//inputFetchStmt.setLong(1, XXX); // Will be filled by loadForms
@ -1746,20 +1741,20 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
List<MForm> connectorConnForms = new ArrayList<MForm>();
List<MForm> frameworkConnForms = new ArrayList<MForm>();
List<MForm> frameworkJobForms = new ArrayList<MForm>();
Map<ConnectorType, List<MForm>> connectorJobForms = new HashMap<ConnectorType, List<MForm>>();
Map<MJob.Type, List<MForm>> connectorJobForms
= new HashMap<MJob.Type, List<MForm>>();
Map<MJob.Type, List<MForm>> frameworkJobForms
= new HashMap<MJob.Type, List<MForm>>();
loadForms(connectorConnForms, connectorJobForms,
formConnectorFetchStmt, inputFetchStmt, 2);
loadConnectorForms(connectorConnForms, connectorJobForms,
formConnectorFetchStmt, inputFetchStmt, 2);
loadForms(frameworkConnForms, frameworkJobForms,
formFrameworkFetchStmt, inputFetchStmt, 2);
MJob job = new MJob(connectorId, connectionId, type,
new MJobForms(type, connectorJobForms.get(type)),
new MJobForms(type, frameworkJobForms.get(type)));
MJob job = new MJob(
fromConnectorId, toConnectorId,
fromConnectionId, toConnectionId,
new MJobForms(connectorJobForms.get(ConnectorType.FROM)),
new MJobForms(connectorJobForms.get(ConnectorType.TO)),
new MJobForms(frameworkJobForms));
job.setPersistenceId(id);
job.setName(name);
@ -1773,8 +1768,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
}
} finally {
closeResultSets(rsJob);
closeStatements(formConnectorFetchStmt,
formFrameworkFetchStmt, inputFetchStmt);
closeStatements(formConnectorFetchStmt, formFrameworkFetchStmt, inputFetchStmt);
}
return jobs;
@ -1791,23 +1785,25 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
* @param type
* @param baseFormStmt
* @param baseInputStmt
* @return short number of forms registered.
* @throws SQLException
*/
private void registerForms(Long connectorId, MJob.Type jobType,
private short registerForms(Long connectorId, ConnectorType connectorType,
List<MForm> forms, String type, PreparedStatement baseFormStmt,
PreparedStatement baseInputStmt)
throws SQLException {
short formIndex = 0;
for (MForm form : forms) {
if(connectorId == null) {
baseFormStmt.setNull(1, Types.BIGINT);
} else {
baseFormStmt.setLong(1, connectorId);
}
if(jobType == null) {
if(connectorType == null) {
baseFormStmt.setNull(2, Types.VARCHAR);
} else {
baseFormStmt.setString(2, jobType.name());
baseFormStmt.setString(2, connectorType.name());
}
baseFormStmt.setString(3, form.getName());
baseFormStmt.setString(4, type);
@ -1830,6 +1826,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
List<MInput<?>> inputs = form.getInputs();
registerFormInputs(formId, inputs, baseInputStmt);
}
return formIndex;
}
/**
@ -1921,7 +1918,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
* @throws SQLException In case of any failure on Derby side
*/
public void loadForms(List<MForm> connectionForms,
Map<MJob.Type, List<MForm>> jobForms,
List<MForm> jobForms,
PreparedStatement formFetchStmt,
PreparedStatement inputFetchStmt,
int formPosition) throws SQLException {
@ -2022,20 +2019,15 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
connectionForms.add(mf);
break;
case JOB:
MJob.Type jobType = MJob.Type.valueOf(operation);
if (!jobForms.containsKey(jobType)) {
jobForms.put(jobType, new ArrayList<MForm>());
}
if (jobForms.get(jobType).size() != formIndex) {
if (jobForms.size() != formIndex) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
"connector-" + formConnectorId
+ "; form: " + mf
+ "; index: " + formIndex
+ "; expected: " + jobForms.get(jobType).size()
+ "; expected: " + jobForms.size()
);
}
jobForms.get(jobType).add(mf);
jobForms.add(mf);
break;
default:
throw new SqoopException(DerbyRepoError.DERBYREPO_0007,
@ -2044,17 +2036,141 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
}
}
public List<MJobForms> convertToJobList(Map<MJob.Type, List<MForm>> l) {
List<MJobForms> ret = new ArrayList<MJobForms>();
/**
* Load forms and corresponding inputs from Derby database.
*
* Use given prepared statements to load all forms and corresponding inputs
* from Derby.
*
* @param connectionForms List of connection forms that will be filled up
* @param jobForms Map with job forms that will be filled up
* @param formFetchStmt Prepared statement for fetching forms
* @param inputFetchStmt Prepare statement for fetching inputs
* @throws SQLException In case of any failure on Derby side
*/
public void loadConnectorForms(List<MForm> connectionForms,
Map<ConnectorType, List<MForm>> jobForms,
PreparedStatement formFetchStmt,
PreparedStatement inputFetchStmt,
int formPosition) throws SQLException {
for (Map.Entry<MJob.Type, List<MForm>> entry : l.entrySet()) {
MJob.Type type = entry.getKey();
List<MForm> forms = entry.getValue();
// Get list of structures from database
ResultSet rsetForm = formFetchStmt.executeQuery();
while (rsetForm.next()) {
long formId = rsetForm.getLong(1);
Long formConnectorId = rsetForm.getLong(2);
String operation = rsetForm.getString(3);
String formName = rsetForm.getString(4);
String formType = rsetForm.getString(5);
int formIndex = rsetForm.getInt(6);
List<MInput<?>> formInputs = new ArrayList<MInput<?>>();
ret.add(new MJobForms(type, forms));
MForm mf = new MForm(formName, formInputs);
mf.setPersistenceId(formId);
inputFetchStmt.setLong(formPosition, formId);
ResultSet rsetInput = inputFetchStmt.executeQuery();
while (rsetInput.next()) {
long inputId = rsetInput.getLong(1);
String inputName = rsetInput.getString(2);
long inputForm = rsetInput.getLong(3);
short inputIndex = rsetInput.getShort(4);
String inputType = rsetInput.getString(5);
boolean inputSensitivity = rsetInput.getBoolean(6);
short inputStrLength = rsetInput.getShort(7);
String inputEnumValues = rsetInput.getString(8);
String value = rsetInput.getString(9);
MInputType mit = MInputType.valueOf(inputType);
MInput input = null;
switch (mit) {
case STRING:
input = new MStringInput(inputName, inputSensitivity, inputStrLength);
break;
case MAP:
input = new MMapInput(inputName, inputSensitivity);
break;
case BOOLEAN:
input = new MBooleanInput(inputName, inputSensitivity);
break;
case INTEGER:
input = new MIntegerInput(inputName, inputSensitivity);
break;
case ENUM:
input = new MEnumInput(inputName, inputSensitivity, inputEnumValues.split(","));
break;
default:
throw new SqoopException(DerbyRepoError.DERBYREPO_0006,
"input-" + inputName + ":" + inputId + ":"
+ "form-" + inputForm + ":" + mit.name());
}
// Set persistent ID
input.setPersistenceId(inputId);
// Set value
if(value == null) {
input.setEmpty();
} else {
input.restoreFromUrlSafeValueString(value);
}
if (mf.getInputs().size() != inputIndex) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0009,
"form: " + mf
+ "; input: " + input
+ "; index: " + inputIndex
+ "; expected: " + mf.getInputs().size()
);
}
mf.getInputs().add(input);
}
if (mf.getInputs().size() == 0) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0008,
"connector-" + formConnectorId
+ "; form: " + mf
);
}
MFormType mft = MFormType.valueOf(formType);
switch (mft) {
case CONNECTION:
if (connectionForms.size() != formIndex) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
"connector-" + formConnectorId
+ "; form: " + mf
+ "; index: " + formIndex
+ "; expected: " + connectionForms.size()
);
}
connectionForms.add(mf);
break;
case JOB:
ConnectorType type = ConnectorType.valueOf(operation);
if (!jobForms.containsKey(type)) {
jobForms.put(type, new ArrayList<MForm>());
}
if (jobForms.get(type).size() != formIndex) {
throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
"connector-" + formConnectorId
+ "; form: " + mf
+ "; index: " + formIndex
+ "; expected: " + jobForms.get(type).size()
);
}
jobForms.get(type).add(mf);
break;
default:
throw new SqoopException(DerbyRepoError.DERBYREPO_0007,
"connector-" + formConnectorId + ":" + mf);
}
}
return ret;
}
private void createInputValues(String query,

Просмотреть файл

@ -144,9 +144,9 @@ public final class DerbySchemaConstants {
public static final String COLUMN_SQB_NAME = "SQB_NAME";
public static final String COLUMN_SQB_TYPE = "SQB_TYPE";
public static final String COLUMN_SQB_FROM_CONNECTION = "SQB_FROM_CONNECTION";
public static final String COLUMN_SQB_CONNECTION = "SQB_CONNECTION";
public static final String COLUMN_SQB_TO_CONNECTION = "SQB_TO_CONNECTION";
public static final String COLUMN_SQB_CREATION_USER = "SQB_CREATION_USER";

Просмотреть файл

@ -286,13 +286,13 @@ public final class DerbySchemaQuery {
public static final String QUERY_CREATE_TABLE_SQ_JOB =
"CREATE TABLE " + TABLE_SQ_JOB + " ("
+ COLUMN_SQB_ID + " BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) PRIMARY KEY, "
+ COLUMN_SQB_CONNECTION + " BIGINT, "
+ COLUMN_SQB_FROM_CONNECTION + " BIGINT, "
+ COLUMN_SQB_TO_CONNECTION + " BIGINT, "
+ COLUMN_SQB_NAME + " VARCHAR(64), "
+ COLUMN_SQB_TYPE + " VARCHAR(64),"
+ COLUMN_SQB_CREATION_DATE + " TIMESTAMP,"
+ COLUMN_SQB_UPDATE_DATE + " TIMESTAMP,"
+ "CONSTRAINT " + CONSTRAINT_SQB_SQN + " "
+ "FOREIGN KEY(" + COLUMN_SQB_CONNECTION + ") "
+ "FOREIGN KEY(" + COLUMN_SQB_FROM_CONNECTION + ") "
+ "REFERENCES " + TABLE_SQ_CONNECTION + " (" + COLUMN_SQN_ID + ")"
+ ")";
@ -702,8 +702,8 @@ public final class DerbySchemaQuery {
public static final String STMT_INSERT_JOB =
"INSERT INTO " + TABLE_SQ_JOB + " ("
+ COLUMN_SQB_NAME + ", "
+ COLUMN_SQB_CONNECTION + ", "
+ COLUMN_SQB_TYPE + ", "
+ COLUMN_SQB_FROM_CONNECTION + ", "
+ COLUMN_SQB_TO_CONNECTION + ", "
+ COLUMN_SQB_ENABLED + ", "
+ COLUMN_SQB_CREATION_USER + ", "
+ COLUMN_SQB_CREATION_DATE + ", "
@ -753,43 +753,49 @@ public final class DerbySchemaQuery {
+ " count(*)"
+ " FROM " + TABLE_SQ_JOB
+ " JOIN " + TABLE_SQ_CONNECTION
+ " ON " + COLUMN_SQB_CONNECTION + " = " + COLUMN_SQN_ID
+ " ON " + COLUMN_SQB_FROM_CONNECTION + " = " + COLUMN_SQN_ID
+ " WHERE " + COLUMN_SQN_ID + " = ? ";
// DML: Select one specific job
public static final String STMT_SELECT_JOB_SINGLE =
"SELECT "
+ COLUMN_SQN_CONNECTOR + ", "
+ COLUMN_SQB_ID + ", "
+ COLUMN_SQB_NAME + ", "
+ COLUMN_SQB_CONNECTION + ", "
+ COLUMN_SQB_TYPE + ", "
+ COLUMN_SQB_ENABLED + ", "
+ COLUMN_SQB_CREATION_USER + ", "
+ COLUMN_SQB_CREATION_DATE + ", "
+ COLUMN_SQB_UPDATE_USER + ", "
+ COLUMN_SQB_UPDATE_DATE
+ " FROM " + TABLE_SQ_JOB
+ "FROM_CONNECTOR." + COLUMN_SQN_CONNECTOR + ", "
+ "TO_CONNECTOR." + COLUMN_SQN_CONNECTOR + ", "
+ "job." + COLUMN_SQB_ID + ", "
+ "job." + COLUMN_SQB_NAME + ", "
+ "job." + COLUMN_SQB_FROM_CONNECTION + ", "
+ "job." + COLUMN_SQB_TO_CONNECTION + ", "
+ "job." + COLUMN_SQB_ENABLED + ", "
+ "job." + COLUMN_SQB_CREATION_USER + ", "
+ "job." + COLUMN_SQB_CREATION_DATE + ", "
+ "job." + COLUMN_SQB_UPDATE_USER + ", "
+ "job." + COLUMN_SQB_UPDATE_DATE
+ " FROM " + TABLE_SQ_JOB + " AS job"
+ " LEFT JOIN " + TABLE_SQ_CONNECTION
+ " ON " + COLUMN_SQB_CONNECTION + " = " + COLUMN_SQN_ID
+ " as FROM_CONNECTOR ON " + COLUMN_SQB_FROM_CONNECTION + " = FROM_CONNECTOR." + COLUMN_SQN_ID
+ " LEFT JOIN " + TABLE_SQ_CONNECTION
+ " as TO_CONNECTOR ON " + COLUMN_SQB_TO_CONNECTION + " = TO_CONNECTOR." + COLUMN_SQN_ID
+ " WHERE " + COLUMN_SQB_ID + " = ?";
// DML: Select all jobs
public static final String STMT_SELECT_JOB_ALL =
"SELECT "
+ COLUMN_SQN_CONNECTOR + ", "
+ COLUMN_SQB_ID + ", "
+ COLUMN_SQB_NAME + ", "
+ COLUMN_SQB_CONNECTION + ", "
+ COLUMN_SQB_TYPE + ", "
+ COLUMN_SQB_ENABLED + ", "
+ COLUMN_SQB_CREATION_USER + ", "
+ COLUMN_SQB_CREATION_DATE + ", "
+ COLUMN_SQB_UPDATE_USER + ", "
+ COLUMN_SQB_UPDATE_DATE
+ " FROM " + TABLE_SQ_JOB
+ "FROM_CONNECTOR." + COLUMN_SQN_CONNECTOR + ", "
+ "TO_CONNECTOR." + COLUMN_SQN_CONNECTOR + ", "
+ "job." + COLUMN_SQB_ID + ", "
+ "job." + COLUMN_SQB_NAME + ", "
+ "job." + COLUMN_SQB_FROM_CONNECTION + ", "
+ "job." + COLUMN_SQB_TO_CONNECTION + ", "
+ "job." + COLUMN_SQB_ENABLED + ", "
+ "job." + COLUMN_SQB_CREATION_USER + ", "
+ "job." + COLUMN_SQB_CREATION_DATE + ", "
+ "job." + COLUMN_SQB_UPDATE_USER + ", "
+ "job." + COLUMN_SQB_UPDATE_DATE
+ " FROM " + TABLE_SQ_JOB + " AS job"
+ " LEFT JOIN " + TABLE_SQ_CONNECTION
+ " ON " + COLUMN_SQB_CONNECTION + " = " + COLUMN_SQN_ID;
+ " as FROM_CONNECTOR ON " + COLUMN_SQB_FROM_CONNECTION + " = FROM_CONNECTOR." + COLUMN_SQN_ID
+ " LEFT JOIN " + TABLE_SQ_CONNECTION
+ " as TO_CONNECTOR ON " + COLUMN_SQB_TO_CONNECTION + " = TO_CONNECTOR." + COLUMN_SQN_ID;
// DML: Select all jobs for a Connector
public static final String STMT_SELECT_ALL_JOBS_FOR_CONNECTOR =
@ -797,8 +803,8 @@ public final class DerbySchemaQuery {
+ COLUMN_SQN_CONNECTOR + ", "
+ COLUMN_SQB_ID + ", "
+ COLUMN_SQB_NAME + ", "
+ COLUMN_SQB_CONNECTION + ", "
+ COLUMN_SQB_TYPE + ", "
+ COLUMN_SQB_FROM_CONNECTION + ", "
+ COLUMN_SQB_TO_CONNECTION + ", "
+ COLUMN_SQB_ENABLED + ", "
+ COLUMN_SQB_CREATION_USER + ", "
+ COLUMN_SQB_CREATION_DATE + ", "
@ -806,7 +812,7 @@ public final class DerbySchemaQuery {
+ COLUMN_SQB_UPDATE_DATE
+ " FROM " + TABLE_SQ_JOB
+ " LEFT JOIN " + TABLE_SQ_CONNECTION
+ " ON " + COLUMN_SQB_CONNECTION + " = " + COLUMN_SQN_ID
+ " ON " + COLUMN_SQB_FROM_CONNECTION + " = " + COLUMN_SQN_ID
+ " AND " + COLUMN_SQN_CONNECTOR + " = ? ";
// DML: Insert new submission

Просмотреть файл

@ -24,8 +24,8 @@ import org.apache.sqoop.connector.ConnectorManager;
import org.apache.sqoop.connector.spi.SqoopConnector;
import org.apache.sqoop.framework.FrameworkManager;
import org.apache.sqoop.json.ConnectionBean;
import org.apache.sqoop.json.ConnectionValidationBean;
import org.apache.sqoop.json.JsonBean;
import org.apache.sqoop.json.ValidationBean;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MConnection;
import org.apache.sqoop.model.MConnectionForms;
@ -204,8 +204,8 @@ public class ConnectionRequestHandler implements RequestHandler {
frameworkValidation.getStatus());
// Return back validations in all cases
ValidationBean outputBean =
new ValidationBean(connectorValidation, frameworkValidation);
ConnectionValidationBean outputBean =
new ConnectionValidationBean(connectorValidation, frameworkValidation);
// If we're good enough let's perform the action
if(finalStatus.canProceed()) {

Просмотреть файл

@ -19,13 +19,14 @@ package org.apache.sqoop.handler;
import org.apache.log4j.Logger;
import org.apache.sqoop.audit.AuditLoggerManager;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.connector.ConnectorManager;
import org.apache.sqoop.connector.spi.SqoopConnector;
import org.apache.sqoop.framework.FrameworkManager;
import org.apache.sqoop.json.JobBean;
import org.apache.sqoop.json.JobValidationBean;
import org.apache.sqoop.json.JsonBean;
import org.apache.sqoop.json.ValidationBean;
import org.apache.sqoop.model.FormUtils;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MJobForms;
@ -163,47 +164,59 @@ public class JobRequestHandler implements RequestHandler {
MJob job = jobs.get(0);
// Verify that user is not trying to spoof us
MJobForms connectorForms
= ConnectorManager.getInstance().getConnectorMetadata(job.getConnectorId())
.getJobForms(job.getType());
MJobForms fromConnectorForms = ConnectorManager.getInstance()
.getConnectorMetadata(job.getConnectorId(ConnectorType.FROM))
.getJobForms(ConnectorType.FROM);
MJobForms toConnectorForms = ConnectorManager.getInstance()
.getConnectorMetadata(job.getConnectorId(ConnectorType.TO))
.getJobForms(ConnectorType.TO);
MJobForms frameworkForms = FrameworkManager.getInstance().getFramework()
.getJobForms(job.getType());
.getJobForms();
if(!connectorForms.equals(job.getConnectorPart())
|| !frameworkForms.equals(job.getFrameworkPart())) {
if(!fromConnectorForms.equals(job.getConnectorPart(ConnectorType.FROM))
|| !frameworkForms.equals(job.getFrameworkPart())
|| !toConnectorForms.equals(job.getConnectorPart(ConnectorType.TO))) {
throw new SqoopException(ServerError.SERVER_0003,
"Detected incorrect form structure");
}
// Responsible connector for this session
SqoopConnector connector =
ConnectorManager.getInstance().getConnector(job.getConnectorId());
SqoopConnector fromConnector =
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.FROM));
SqoopConnector toConnector =
ConnectorManager.getInstance().getConnector(job.getConnectorId(ConnectorType.TO));
// Get validator objects
Validator connectorValidator = connector.getValidator();
Validator fromConnectorValidator = fromConnector.getValidator();
Validator frameworkValidator = FrameworkManager.getInstance().getValidator();
Validator toConnectorValidator = toConnector.getValidator();
// We need translate forms to configuration objects
Object connectorConfig = ClassUtils.instantiate(
connector.getJobConfigurationClass(job.getType()));
Object fromConnectorConfig = ClassUtils.instantiate(
fromConnector.getJobConfigurationClass(ConnectorType.FROM));
Object frameworkConfig = ClassUtils.instantiate(
FrameworkManager.getInstance().getJobConfigurationClass(job.getType()));
FrameworkManager.getInstance().getJobConfigurationClass());
Object toConnectorConfig = ClassUtils.instantiate(
toConnector.getJobConfigurationClass(ConnectorType.TO));
FormUtils.fromForms(job.getConnectorPart().getForms(), connectorConfig);
FormUtils.fromForms(job.getConnectorPart(ConnectorType.FROM).getForms(), fromConnectorConfig);
FormUtils.fromForms(job.getFrameworkPart().getForms(), frameworkConfig);
FormUtils.fromForms(job.getConnectorPart(ConnectorType.TO).getForms(), toConnectorConfig);
// Validate both parts
Validation connectorValidation =
connectorValidator.validateJob(job.getType(), connectorConfig);
// Validate all parts
Validation fromConnectorValidation =
fromConnectorValidator.validateJob(fromConnectorConfig);
Validation frameworkValidation =
frameworkValidator.validateJob(job.getType(), frameworkConfig);
frameworkValidator.validateJob(frameworkConfig);
Validation toConnectorValidation =
toConnectorValidator.validateJob(toConnectorConfig);
Status finalStatus = Status.getWorstStatus(connectorValidation.getStatus(),
frameworkValidation.getStatus());
Status finalStatus = Status.getWorstStatus(fromConnectorValidation.getStatus(),
frameworkValidation.getStatus(), toConnectorValidation.getStatus());
// Return back validations in all cases
ValidationBean outputBean =
new ValidationBean(connectorValidation, frameworkValidation);
JobValidationBean outputBean =
new JobValidationBean(fromConnectorValidation, frameworkValidation, toConnectorValidation);
// If we're good enough let's perform the action
if(finalStatus.canProceed()) {
@ -247,8 +260,9 @@ public class JobRequestHandler implements RequestHandler {
bean = new JobBean(jobs);
// Add associated resources into the bean
// @TODO(Abe): From/To.
for( MJob job : jobs) {
long connectorId = job.getConnectorId();
long connectorId = job.getConnectorId(ConnectorType.FROM);
if(!bean.hasConnectorBundle(connectorId)) {
bean.addConnectorBundle(connectorId,
ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
@ -258,7 +272,8 @@ public class JobRequestHandler implements RequestHandler {
long jid = Long.valueOf(sjid);
MJob job = repository.findJob(jid);
long connectorId = job.getConnectorId();
// @TODO(Abe): From/To
long connectorId = job.getConnectorId(ConnectorType.FROM);
bean = new JobBean(job);

Просмотреть файл

@ -20,6 +20,7 @@ package org.apache.sqoop.shell;
import jline.ConsoleReader;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.OptionBuilder;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MPersistableEntity;
import org.apache.sqoop.shell.core.Constants;
@ -70,8 +71,11 @@ public class CloneJobFunction extends SqoopFunction {
MJob job = client.getJob(jobId);
job.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
ResourceBundle connectorBundle = client.getResourceBundle(job.getConnectorId());
ResourceBundle fromConnectorBundle = client.getResourceBundle(
job.getConnectorId(ConnectorType.FROM));
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
ResourceBundle toConnectorBundle = client.getResourceBundle(
job.getConnectorId(ConnectorType.TO));
Status status = Status.FINE;
@ -88,7 +92,7 @@ public class CloneJobFunction extends SqoopFunction {
}
// Fill in data from user
if(!fillJob(reader, job, connectorBundle, frameworkBundle)) {
if(!fillJob(reader, job, fromConnectorBundle, frameworkBundle, toConnectorBundle)) {
return null;
}

Просмотреть файл

@ -20,6 +20,7 @@ package org.apache.sqoop.shell;
import jline.ConsoleReader;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.OptionBuilder;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.shell.core.Constants;
import org.apache.sqoop.shell.utils.FormDisplayer;
@ -43,26 +44,26 @@ public class CreateJobFunction extends SqoopFunction {
public CreateJobFunction() {
this.addOption(OptionBuilder
.withDescription(resourceString(Constants.RES_PROMPT_CONN_ID))
.withLongOpt(Constants.OPT_XID)
.withLongOpt(Constants.OPT_FXID)
.hasArg()
.create(Constants.OPT_XID_CHAR)
.create(Constants.OPT_FXID_CHAR)
);
this.addOption(OptionBuilder
.withDescription(resourceString(Constants.RES_PROMPT_JOB_TYPE))
.withLongOpt(Constants.OPT_TYPE)
.withDescription(resourceString(Constants.RES_PROMPT_CONN_ID))
.withLongOpt(Constants.OPT_TXID)
.hasArg()
.create(Constants.OPT_TYPE_CHAR)
.create(Constants.OPT_TXID_CHAR)
);
}
@Override
public boolean validateArgs(CommandLine line) {
if (!line.hasOption(Constants.OPT_XID)) {
printlnResource(Constants.RES_ARGS_XID_MISSING);
if (!line.hasOption(Constants.OPT_FXID)) {
printlnResource(Constants.RES_ARGS_FXID_MISSING);
return false;
}
if (!line.hasOption(Constants.OPT_TYPE)) {
printlnResource(Constants.RES_ARGS_TYPE_MISSING);
if (!line.hasOption(Constants.OPT_TXID)) {
printlnResource(Constants.RES_ARGS_TXID_MISSING);
return false;
}
return true;
@ -71,19 +72,23 @@ public class CreateJobFunction extends SqoopFunction {
@Override
@SuppressWarnings("unchecked")
public Object executeFunction(CommandLine line, boolean isInteractive) throws IOException {
return createJob(getLong(line, Constants.OPT_XID),
line.getOptionValue(Constants.OPT_TYPE),
return createJob(getLong(line, Constants.OPT_FXID),
getLong(line, Constants.OPT_TXID),
line.getArgList(),
isInteractive);
}
private Status createJob(Long connectionId, String type, List<String> args, boolean isInteractive) throws IOException {
printlnResource(Constants.RES_CREATE_CREATING_JOB, connectionId);
private Status createJob(Long fromConnectionId, Long toConnectionId, List<String> args, boolean isInteractive) throws IOException {
printlnResource(Constants.RES_CREATE_CREATING_JOB, fromConnectionId, toConnectionId);
ConsoleReader reader = new ConsoleReader();
MJob job = client.newJob(connectionId, MJob.Type.valueOf(type.toUpperCase()));
MJob job = client.newJob(fromConnectionId, toConnectionId);
ResourceBundle connectorBundle = client.getResourceBundle(job.getConnectorId());
// @TODO(Abe): From/To.
ResourceBundle fromConnectorBundle = client.getResourceBundle(
job.getConnectorId(ConnectorType.FROM));
ResourceBundle toConnectorBundle = client.getResourceBundle(
job.getConnectorId(ConnectorType.TO));
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
Status status = Status.FINE;
@ -98,7 +103,7 @@ public class CreateJobFunction extends SqoopFunction {
}
// Fill in data from user
if(!fillJob(reader, job, connectorBundle, frameworkBundle)) {
if(!fillJob(reader, job, fromConnectorBundle, frameworkBundle, toConnectorBundle)) {
return null;
}

Просмотреть файл

@ -40,7 +40,7 @@ public class DeleteConnectionFunction extends SqoopFunction {
@Override
public boolean validateArgs(CommandLine line) {
if (!line.hasOption(Constants.OPT_XID)) {
if (!line.hasOption(Constants.OPT_FXID)) {
printlnResource(Constants.RES_ARGS_XID_MISSING);
return false;
}

Просмотреть файл

@ -42,9 +42,9 @@ public class ShowConnectionFunction extends SqoopFunction {
.withDescription(resourceString(Constants.RES_SHOW_PROMPT_DISPLAY_ALL_CONNS))
.withLongOpt(Constants.OPT_ALL)
.create(Constants.OPT_ALL_CHAR));
this.addOption(OptionBuilder.hasArg().withArgName(Constants.OPT_XID)
this.addOption(OptionBuilder.hasArg().withArgName(Constants.OPT_FXID)
.withDescription(resourceString(Constants.RES_SHOW_PROMPT_DISPLAY_CONN_XID))
.withLongOpt(Constants.OPT_XID)
.withLongOpt(Constants.OPT_FXID)
.create(Constants.OPT_XID_CHAR));
}
@ -52,8 +52,8 @@ public class ShowConnectionFunction extends SqoopFunction {
public Object executeFunction(CommandLine line, boolean isInteractive) {
if (line.hasOption(Constants.OPT_ALL)) {
showConnections();
} else if (line.hasOption(Constants.OPT_XID)) {
showConnection(getLong(line, Constants.OPT_XID));
} else if (line.hasOption(Constants.OPT_FXID)) {
showConnection(getLong(line, Constants.OPT_FXID));
} else {
showSummary();
}

Просмотреть файл

@ -19,6 +19,7 @@ package org.apache.sqoop.shell;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.OptionBuilder;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.shell.core.Constants;
import org.apache.sqoop.shell.utils.TableDisplayer;
@ -67,25 +68,27 @@ public class ShowJobFunction extends SqoopFunction {
List<String> header = new LinkedList<String>();
header.add(resourceString(Constants.RES_TABLE_HEADER_ID));
header.add(resourceString(Constants.RES_TABLE_HEADER_NAME));
header.add(resourceString(Constants.RES_TABLE_HEADER_TYPE));
header.add(resourceString(Constants.RES_TABLE_HEADER_CONNECTOR));
header.add(resourceString(Constants.RES_TABLE_HEADER_FROM_CONNECTOR));
header.add(resourceString(Constants.RES_TABLE_HEADER_TO_CONNECTOR));
header.add(resourceString(Constants.RES_TABLE_HEADER_ENABLED));
List<String> ids = new LinkedList<String>();
List<String> names = new LinkedList<String>();
List<String> types = new LinkedList<String>();
List<String> connectors = new LinkedList<String>();
List<String> fromConnectors = new LinkedList<String>();
List<String> toConnectors = new LinkedList<String>();
List<String> availabilities = new LinkedList<String>();
for(MJob job : jobs) {
ids.add(String.valueOf(job.getPersistenceId()));
names.add(job.getName());
types.add(job.getType().toString());
connectors.add(String.valueOf(job.getConnectorId()));
fromConnectors.add(String.valueOf(
job.getConnectorId(ConnectorType.FROM)));
toConnectors.add(String.valueOf(
job.getConnectorId(ConnectorType.TO)));
availabilities.add(String.valueOf(job.getEnabled()));
}
TableDisplayer.display(header, ids, names, types, connectors, availabilities);
TableDisplayer.display(header, ids, names, fromConnectors, toConnectors, availabilities);
}
private void showJobs() {
@ -118,13 +121,15 @@ public class ShowJobFunction extends SqoopFunction {
formatter.format(job.getLastUpdateDate())
);
printlnResource(Constants.RES_SHOW_PROMPT_JOB_XID_CID_INFO,
job.getConnectionId(),
job.getConnectorId());
job.getConnectionId(ConnectorType.FROM),
job.getConnectorId(ConnectorType.FROM));
// Display connector part
displayForms(job.getConnectorPart().getForms(),
client.getResourceBundle(job.getConnectorId()));
displayForms(job.getConnectorPart(ConnectorType.FROM).getForms(),
client.getResourceBundle(job.getConnectorId(ConnectorType.FROM)));
displayForms(job.getFrameworkPart().getForms(),
client.getFrameworkResourceBundle());
displayForms(job.getConnectorPart(ConnectorType.TO).getForms(),
client.getResourceBundle(job.getConnectorId(ConnectorType.TO)));
}
}

Просмотреть файл

@ -20,6 +20,7 @@ package org.apache.sqoop.shell;
import jline.ConsoleReader;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.OptionBuilder;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.shell.core.Constants;
import org.apache.sqoop.shell.utils.FormDisplayer;
@ -70,8 +71,11 @@ public class UpdateJobFunction extends SqoopFunction {
MJob job = client.getJob(jobId);
ResourceBundle connectorBundle = client.getResourceBundle(job.getConnectorId());
ResourceBundle fromConnectorBundle = client.getResourceBundle(
job.getConnectorId(ConnectorType.FROM));
ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
ResourceBundle toConnectorBundle = client.getResourceBundle(
job.getConnectorId(ConnectorType.TO));
Status status = Status.FINE;
@ -85,7 +89,7 @@ public class UpdateJobFunction extends SqoopFunction {
}
// Fill in data from user
if(!fillJob(reader, job, connectorBundle, frameworkBundle)) {
if(!fillJob(reader, job, fromConnectorBundle, frameworkBundle, toConnectorBundle)) {
return status;
}

Просмотреть файл

@ -35,6 +35,8 @@ public class Constants {
// Options
public static final String OPT_XID = "xid";
public static final String OPT_FXID = "fxid";
public static final String OPT_TXID = "txid";
public static final String OPT_ALL = "all";
public static final String OPT_JID = "jid";
public static final String OPT_CID = "cid";
@ -54,6 +56,8 @@ public class Constants {
public static final String OPT_DETAIL = "detail";
public static final char OPT_XID_CHAR = 'x';
public static final char OPT_FXID_CHAR = 'f';
public static final char OPT_TXID_CHAR = 't';
public static final char OPT_ALL_CHAR = 'a';
public static final char OPT_JID_CHAR = 'j';
public static final char OPT_CID_CHAR = 'c';
@ -143,12 +147,14 @@ public class Constants {
"args.function.unknown";
public static final String RES_ARGS_XID_MISSING =
"args.xid_missing";
public static final String RES_ARGS_FXID_MISSING =
"args.fxid_missing";
public static final String RES_ARGS_TXID_MISSING =
"args.txid_missing";
public static final String RES_ARGS_JID_MISSING =
"args.jid_missing";
public static final String RES_ARGS_CID_MISSING =
"args.cid_missing";
public static final String RES_ARGS_TYPE_MISSING =
"args.type_missing";
public static final String RES_ARGS_NAME_MISSING =
"args.name_missing";
public static final String RES_ARGS_VALUE_MISSING =
@ -160,8 +166,6 @@ public class Constants {
"prompt.job_id";
public static final String RES_CONNECTOR_ID =
"prompt.connector_id";
public static final String RES_PROMPT_JOB_TYPE =
"prompt.job_type";
public static final String RES_PROMPT_UPDATE_CONN_METADATA =
"prompt.update_conn_metadata";
public static final String RES_PROMPT_UPDATE_JOB_METADATA =
@ -375,10 +379,12 @@ public class Constants {
"table.header.version";
public static final String RES_TABLE_HEADER_CLASS =
"table.header.class";
public static final String RES_TABLE_HEADER_TYPE =
"table.header.type";
public static final String RES_TABLE_HEADER_CONNECTOR =
"table.header.connector";
public static final String RES_TABLE_HEADER_FROM_CONNECTOR =
"table.header.connector.from";
public static final String RES_TABLE_HEADER_TO_CONNECTOR =
"table.header.connector.to";
public static final String RES_TABLE_HEADER_JOB_ID =
"table.header.jid";
public static final String RES_TABLE_HEADER_EXTERNAL_ID =
@ -390,14 +396,10 @@ public class Constants {
public static final String RES_TABLE_HEADER_ENABLED =
"table.header.enabled";
public static final String RES_FORMDISPLAYER_SUPPORTED_JOBTYPE =
"formdisplayer.supported_job_types";
public static final String RES_FORMDISPLAYER_CONNECTION =
"formdisplayer.connection";
public static final String RES_FORMDISPLAYER_JOB =
"formdisplayer.job";
public static final String RES_FORMDISPLAYER_FORM_JOBTYPE =
"formdisplayer.forms_jobtype";
public static final String RES_FORMDISPLAYER_FORM =
"formdisplayer.form";
public static final String RES_FORMDISPLAYER_NAME =

Просмотреть файл

@ -18,9 +18,11 @@
package org.apache.sqoop.shell.utils;
import org.apache.commons.lang.StringUtils;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MAccountableEntity;
import org.apache.sqoop.model.MBooleanInput;
import org.apache.sqoop.model.MConnection;
import org.apache.sqoop.model.MConnector;
import org.apache.sqoop.model.MEnumInput;
import org.apache.sqoop.model.MForm;
import org.apache.sqoop.model.MFramework;
@ -28,7 +30,6 @@ import org.apache.sqoop.model.MInput;
import org.apache.sqoop.model.MInputType;
import org.apache.sqoop.model.MIntegerInput;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.model.MJobForms;
import org.apache.sqoop.model.MMapInput;
import org.apache.sqoop.model.MStringInput;
import org.apache.sqoop.shell.core.Constants;
@ -49,21 +50,34 @@ public final class FormDisplayer {
public static void displayFormMetadataDetails(MFramework framework,
ResourceBundle bundle) {
print(" %s: ", resourceString(Constants.RES_FORMDISPLAYER_SUPPORTED_JOBTYPE));
println(framework.getAllJobsForms().keySet().toString());
displayFormsMetadata(
framework.getConnectionForms().getForms(),
resourceString(Constants.RES_FORMDISPLAYER_CONNECTION),
bundle);
for (MJobForms jobForms : framework.getAllJobsForms().values()) {
print(" %s ", resourceString(Constants.RES_FORMDISPLAYER_FORM_JOBTYPE));
print(jobForms.getType().name());
println(":");
displayFormsMetadata(
framework.getJobForms().getForms(),
resourceString(Constants.RES_FORMDISPLAYER_JOB),
bundle);
}
displayFormsMetadata(jobForms.getForms(), resourceString(Constants.RES_FORMDISPLAYER_JOB), bundle);
}
public static void displayFormMetadataDetails(MConnector connector,
ResourceBundle bundle) {
displayFormsMetadata(
connector.getConnectionForms().getForms(),
resourceString(Constants.RES_FORMDISPLAYER_CONNECTION),
bundle);
// @TODO(Abe): Validate From/To output is correct.
displayFormsMetadata(
connector.getJobForms(ConnectorType.FROM).getForms(),
resourceString(Constants.RES_FORMDISPLAYER_JOB),
bundle);
displayFormsMetadata(
connector.getJobForms(ConnectorType.TO).getForms(),
resourceString(Constants.RES_FORMDISPLAYER_JOB),
bundle);
}
public static void displayFormsMetadata(List<MForm> forms,
@ -139,8 +153,9 @@ public final class FormDisplayer {
formList.addAll(connection.getFrameworkPart().getForms());
} else if(entity instanceof MJob) {
MJob job = (MJob) entity;
formList.addAll(job.getConnectorPart().getForms());
formList.addAll(job.getConnectorPart(ConnectorType.FROM).getForms());
formList.addAll(job.getFrameworkPart().getForms());
formList.addAll(job.getConnectorPart(ConnectorType.TO).getForms());
}
for(MForm form : formList) {
if(form.getValidationStatus() == Status.ACCEPTABLE) {

Просмотреть файл

@ -21,6 +21,7 @@ import jline.ConsoleReader;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.lang.StringUtils;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MBooleanInput;
import org.apache.sqoop.model.MConnection;
import org.apache.sqoop.model.MEnumInput;
@ -55,7 +56,7 @@ public final class FormFiller {
/**
* Fill job object based on CLI options.
*
* @param reader Associated console reader object
* @param line Associated console reader object
* @param job Job that user is suppose to fill in
* @return True if we filled all inputs, false if user has stopped processing
* @throws IOException
@ -68,7 +69,7 @@ public final class FormFiller {
// Fill in data from user
return fillForms(line,
job.getConnectorPart().getForms(),
job.getConnectorPart(ConnectorType.FROM).getForms(),
job.getFrameworkPart().getForms());
}
@ -77,25 +78,28 @@ public final class FormFiller {
*
* @param reader Associated console reader object
* @param job Job that user is suppose to fill in
* @param connectorBundle Connector resource bundle
* @param fromConnectorBundle Connector resource bundle
* @param frameworkBundle Framework resource bundle
* @return True if we filled all inputs, false if user has stopped processing
* @throws IOException
*/
public static boolean fillJob(ConsoleReader reader,
MJob job,
ResourceBundle connectorBundle,
ResourceBundle frameworkBundle)
ResourceBundle fromConnectorBundle,
ResourceBundle frameworkBundle,
ResourceBundle toConnectorBundle)
throws IOException {
job.setName(getName(reader, job.getName()));
// Fill in data from user
return fillForms(reader,
job.getConnectorPart().getForms(),
connectorBundle,
job.getConnectorPart(ConnectorType.FROM).getForms(),
fromConnectorBundle,
job.getFrameworkPart().getForms(),
frameworkBundle);
frameworkBundle,
job.getConnectorPart(ConnectorType.TO).getForms(),
toConnectorBundle);
}
/**
@ -387,8 +391,7 @@ public final class FormFiller {
List<MForm> connectorForms,
ResourceBundle connectorBundle,
List<MForm> frameworkForms,
ResourceBundle frameworkBundle
) throws IOException {
ResourceBundle frameworkBundle) throws IOException {
// Query connector forms
@ -400,6 +403,32 @@ public final class FormFiller {
if(!fillForms(frameworkForms, reader, frameworkBundle)) {
return false;
}
return true;
}
public static boolean fillForms(ConsoleReader reader,
List<MForm> fromConnectorForms,
ResourceBundle fromConnectorBundle,
List<MForm> frameworkForms,
ResourceBundle frameworkBundle,
List<MForm> toConnectorForms,
ResourceBundle toConnectorBundle) throws IOException {
// From connector forms
if(!fillForms(fromConnectorForms, reader, fromConnectorBundle)) {
return false;
}
// Query framework forms
if(!fillForms(frameworkForms, reader, frameworkBundle)) {
return false;
}
// To connector forms
if(!fillForms(toConnectorForms, reader, toConnectorBundle)) {
return false;
}
return true;
}
@ -880,7 +909,7 @@ public final class FormFiller {
}
public static void printJobValidationMessages(MJob job) {
for (MForm form : job.getConnectorPart().getForms()) {
for (MForm form : job.getConnectorPart(ConnectorType.FROM).getForms()) {
for (MInput<?> input : form.getInputs()) {
printValidationMessage(input, true);
}
@ -890,6 +919,11 @@ public final class FormFiller {
printValidationMessage(input, true);
}
}
for (MForm form : job.getConnectorPart(ConnectorType.TO).getForms()) {
for (MInput<?> input : form.getInputs()) {
printValidationMessage(input, true);
}
}
}
private FormFiller() {

Просмотреть файл

@ -19,6 +19,7 @@ package org.apache.sqoop.shell.utils;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.model.MJob;
/**
@ -34,11 +35,14 @@ public class JobDynamicFormOptions extends DynamicFormOptions<MJob> {
.withLongOpt("name")
.hasArg()
.create());
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart().getForms())) {
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart(ConnectorType.FROM).getForms())) {
this.addOption(option);
}
for (Option option : FormOptions.getFormsOptions("framework", job.getFrameworkPart().getForms())) {
this.addOption(option);
}
for (Option option : FormOptions.getFormsOptions("connector", job.getConnectorPart(ConnectorType.TO).getForms())) {
this.addOption(option);
}
}
}

Просмотреть файл

@ -30,9 +30,10 @@ object-name.help = Non unique name of the entity to help you remember \
#
args.function.unknown = The specified function "{0}" is not recognized.
args.xid_missing = Required argument --xid is missing.
args.fxid_missing = Required argument --fxid is missing.
args.txid_missing = Required argument --txid is missing.
args.jid_missing = Required argument --jid is missing.
args.cid_missing = Required argument --cid is missing.
args.type_missing = Required argument --type is missing.
args.name_missing = Required argument --name is missing.
args.value_missing = Required argument --value is missing.
@ -79,7 +80,7 @@ create.job_successful = New job was successfully created with validation \
status {0} and persistent id {1}
## Creating messages
create.creating_conn = Creating connection for connector with id {0}
create.creating_job = Creating job for connection with id {0}
create.creating_job = Creating job for connections with id {0} and {1}
#
# Delete command
@ -193,8 +194,9 @@ table.header.id = Id
table.header.name = Name
table.header.version = Version
table.header.class = Class
table.header.type = Type
table.header.connector = Connector
table.header.connector.from = From Connector
table.header.connector.to = To Connector
table.header.jid = Job Id
table.header.eid = External Id
table.header.status = Status
@ -205,7 +207,6 @@ table.header.enabled = Enabled
formdisplayer.supported_job_types = Supported job types
formdisplayer.connection = Connection
formdisplayer.job = Job
formdisplayer.forms_jobtype = Forms for job type
formdisplayer.form = form
formdisplayer.name = Name
formdisplayer.label = Label

Просмотреть файл

@ -20,11 +20,11 @@ package org.apache.sqoop.connector.spi;
import java.util.Locale;
import java.util.ResourceBundle;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.connector.idf.CSVIntermediateDataFormat;
import org.apache.sqoop.connector.idf.IntermediateDataFormat;
import org.apache.sqoop.job.etl.Exporter;
import org.apache.sqoop.job.etl.Importer;
import org.apache.sqoop.model.MJob;
import org.apache.sqoop.job.etl.From;
import org.apache.sqoop.job.etl.To;
import org.apache.sqoop.validation.Validator;
/**
@ -53,17 +53,17 @@ public abstract class SqoopConnector {
/**
* @return Get job configuration class for given type or null if not supported
*/
public abstract Class getJobConfigurationClass(MJob.Type jobType);
public abstract Class getJobConfigurationClass(ConnectorType jobType);
/**
* @return an <tt>Importer</tt> that provides classes for performing import.
* @return an <tt>From</tt> that provides classes for performing import.
*/
public abstract Importer getImporter();
public abstract From getFrom();
/**
* @return an <tt>Exporter</tt> that provides classes for performing export.
* @return an <tt>To</tt> that provides classes for performing export.
*/
public abstract Exporter getExporter();
public abstract To getTo();
/**
* Returns validation object that Sqoop framework can use to validate user

Просмотреть файл

@ -26,15 +26,15 @@ package org.apache.sqoop.job.etl;
* -> (framework-defined steps)
* -> Destroyer
*/
public class Importer extends CallbackBase {
public class From extends CallbackBase {
private Class<? extends Partitioner> partitioner;
private Class<? extends Extractor> extractor;
public Importer(Class<? extends Initializer> initializer,
Class<? extends Partitioner> partitioner,
Class<? extends Extractor> extractor,
Class<? extends Destroyer> destroyer) {
public From(Class<? extends Initializer> initializer,
Class<? extends Partitioner> partitioner,
Class<? extends Extractor> extractor,
Class<? extends Destroyer> destroyer) {
super(initializer, destroyer);
this.partitioner = partitioner;
this.extractor = extractor;

Просмотреть файл

@ -25,15 +25,15 @@ package org.apache.sqoop.job.etl;
* -> Loader
* -> Destroyer
*/
public class Exporter extends CallbackBase {
public class To extends CallbackBase {
private Class<? extends Loader> loader;
public Exporter(
public To(
Class<? extends Initializer> initializer,
Class<? extends Loader> loader,
Class<? extends Destroyer> destroyer
) {
) {
super(initializer, destroyer);
this.loader = loader;
}

Просмотреть файл

@ -40,11 +40,10 @@ public class Validator {
/**
* Validate configuration object for job .
*
* @param type Type of jobs that being validated
* @param jobConfiguration Job to be validated
* @return Validation status
*/
public Validation validateJob(MJob.Type type, Object jobConfiguration) {
public Validation validateJob(Object jobConfiguration) {
return new Validation(EmptyClass.class);
}

Просмотреть файл

@ -19,15 +19,14 @@ package org.apache.sqoop.submission.mapreduce;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobID;
import org.apache.hadoop.mapred.JobStatus;
import org.apache.hadoop.mapred.RunningJob;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.log4j.Logger;
import org.apache.sqoop.common.ConnectorType;
import org.apache.sqoop.common.MapContext;
import org.apache.sqoop.common.SqoopException;
import org.apache.sqoop.execution.mapreduce.MRSubmissionRequest;
@ -155,9 +154,6 @@ public class MapreduceSubmissionEngine extends SubmissionEngine {
// Clone global configuration
Configuration configuration = new Configuration(globalConfiguration);
// Serialize job type as it will be needed by underlying execution engine
ConfigurationUtils.setJobType(configuration, request.getJobType());
// Serialize framework context into job configuration
for(Map.Entry<String, String> entry: request.getFrameworkContext()) {
if (entry.getValue() == null) {
@ -168,16 +164,26 @@ public class MapreduceSubmissionEngine extends SubmissionEngine {
}
// Serialize connector context as a sub namespace
for(Map.Entry<String, String> entry :request.getConnectorContext()) {
for(Map.Entry<String, String> entry : request.getConnectorContext(ConnectorType.FROM)) {
if (entry.getValue() == null) {
LOG.warn("Ignoring null connector context value for key " + entry.getKey());
continue;
}
configuration.set(
JobConstants.PREFIX_CONNECTOR_CONTEXT + entry.getKey(),
JobConstants.PREFIX_CONNECTOR_FROM_CONTEXT + entry.getKey(),
entry.getValue());
}
for(Map.Entry<String, String> entry : request.getConnectorContext(ConnectorType.TO)) {
if (entry.getValue() == null) {
LOG.warn("Ignoring null connector context value for key " + entry.getKey());
continue;
}
configuration.set(
JobConstants.PREFIX_CONNECTOR_TO_CONTEXT + entry.getKey(),
entry.getValue());
}
// Set up notification URL if it's available
if(request.getNotificationUrl() != null) {
configuration.set("job.end.notification.url", request.getNotificationUrl());
@ -194,9 +200,12 @@ public class MapreduceSubmissionEngine extends SubmissionEngine {
Job job = new Job(configuration);
// And finally put all configuration objects to credentials cache
ConfigurationUtils.setConfigConnectorConnection(job, request.getConfigConnectorConnection());
ConfigurationUtils.setConfigConnectorJob(job, request.getConfigConnectorJob());
ConfigurationUtils.setConfigFrameworkConnection(job, request.getConfigFrameworkConnection());
ConfigurationUtils.setConnectorConnectionConfig(ConnectorType.FROM, job, request.getConnectorConnectionConfig(ConnectorType.FROM));
ConfigurationUtils.setConnectorJobConfig(ConnectorType.FROM, job, request.getConnectorJobConfig(ConnectorType.FROM));
ConfigurationUtils.setConnectorConnectionConfig(ConnectorType.TO, job, request.getConnectorConnectionConfig(ConnectorType.TO));
ConfigurationUtils.setConnectorJobConfig(ConnectorType.TO, job, request.getConnectorJobConfig(ConnectorType.TO));
ConfigurationUtils.setFrameworkConnectionConfig(ConnectorType.FROM, job, request.getFrameworkConnectionConfig(ConnectorType.FROM));
ConfigurationUtils.setFrameworkConnectionConfig(ConnectorType.TO, job, request.getFrameworkConnectionConfig(ConnectorType.TO));
ConfigurationUtils.setConfigFrameworkJob(job, request.getConfigFrameworkJob());
ConfigurationUtils.setConnectorSchema(job, request.getSummary().getConnectorSchema());
@ -212,11 +221,6 @@ public class MapreduceSubmissionEngine extends SubmissionEngine {
job.setMapOutputKeyClass(request.getMapOutputKeyClass());
job.setMapOutputValueClass(request.getMapOutputValueClass());
String outputDirectory = request.getOutputDirectory();
if(outputDirectory != null) {
FileOutputFormat.setOutputPath(job, new Path(outputDirectory));
}
// Set number of reducers as number of configured loaders or suppress
// reduce phase entirely if loaders are not set at all.
if(request.getLoaders() != null) {