mirror of
https://github.com/apache/sqoop.git
synced 2025-05-02 19:50:39 +08:00
SQOOP-3419: Refactor S3 tests for easier extensibility
(Szabolcs Vasas)
This commit is contained in:
parent
4a22691f45
commit
032b828370
78
src/test/org/apache/sqoop/cloud/AbstractTestAvroImport.java
Normal file
78
src/test/org/apache/sqoop/cloud/AbstractTestAvroImport.java
Normal file
@ -0,0 +1,78 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.AvroTestUtils;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestAvroImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestAvroImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestAvroImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsAvroDataFileWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws IOException {
|
||||||
|
String[] args = getArgsWithAsAvroDataFileOption();
|
||||||
|
runImport(args);
|
||||||
|
AvroTestUtils.verify(getDataSet().getExpectedAvroOutput(), fileSystemRule.getCloudFileSystem().getConf(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsAvroDataFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
||||||
|
String[] args = getArgsWithAsAvroDataFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getArgsWithAsAvroDataFileAndDeleteTargetDirOption();
|
||||||
|
runImport(args);
|
||||||
|
AvroTestUtils.verify(getDataSet().getExpectedAvroOutput(), fileSystemRule.getCloudFileSystem().getConf(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsAvroDataFileWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
||||||
|
String[] args = getArgsWithAsAvroDataFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsAvroDataFileOption() {
|
||||||
|
return getArgsForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-avrodatafile");
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsAvroDataFileAndDeleteTargetDirOption() {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-avrodatafile");
|
||||||
|
builder.withOption("delete-target-dir");
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,108 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.tool.BaseSqoopTool.FMT_PARQUETFILE_ARG;
|
||||||
|
import static org.apache.sqoop.tool.BaseSqoopTool.FMT_TEXTFILE_ARG;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudTestDataSet;
|
||||||
|
import org.apache.sqoop.hive.minicluster.HiveMiniCluster;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.HiveServer2TestUtil;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class AbstractTestExternalHiveTableImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
private static CloudTestDataSet dataSet = new CloudTestDataSet();
|
||||||
|
|
||||||
|
@Parameterized.Parameters(name = "fileFormatArg = {0}")
|
||||||
|
public static Iterable<? extends Object> parameters() {
|
||||||
|
return Arrays.asList(new Object[]{FMT_TEXTFILE_ARG, dataSet.getExpectedTextOutputAsList()},
|
||||||
|
new Object[]{FMT_PARQUETFILE_ARG, dataSet.getExpectedParquetOutput()});
|
||||||
|
}
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestExternalHiveTableImport.class.getName());
|
||||||
|
|
||||||
|
private String fileFormatArg;
|
||||||
|
|
||||||
|
private List<String> expectedResult;
|
||||||
|
|
||||||
|
protected AbstractTestExternalHiveTableImport(CloudCredentialsRule credentialsRule, String fileFormatArg, List<String> expectedResult) {
|
||||||
|
super(credentialsRule);
|
||||||
|
this.fileFormatArg = fileFormatArg;
|
||||||
|
this.expectedResult = expectedResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
private HiveMiniCluster hiveMiniCluster;
|
||||||
|
|
||||||
|
private HiveServer2TestUtil hiveServer2TestUtil;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
super.setUp();
|
||||||
|
hiveMiniCluster = createCloudBasedHiveMiniCluster();
|
||||||
|
hiveServer2TestUtil = new HiveServer2TestUtil(hiveMiniCluster.getUrl());
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void stopMiniCluster() {
|
||||||
|
if (hiveMiniCluster != null) {
|
||||||
|
hiveMiniCluster.stop();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportIntoExternalHiveTable() throws IOException {
|
||||||
|
String[] args = getExternalHiveTableImportArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> rows = hiveServer2TestUtil.loadCsvRowsFromTable(getTableName());
|
||||||
|
assertEquals(rows, expectedResult);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateAndImportIntoExternalHiveTable() throws IOException {
|
||||||
|
String[] args = getExternalHiveTableImportArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> rows = hiveServer2TestUtil.loadCsvRowsFromTable(getHiveExternalTableName());
|
||||||
|
assertEquals(rows, expectedResult);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getExternalHiveTableImportArgs(boolean createHiveTable) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), fileFormatArg);
|
||||||
|
builder = addExternalHiveTableImportArgs(builder, hiveMiniCluster.getUrl(), fileSystemRule.getExternalTableDirPath().toString());
|
||||||
|
if (createHiveTable) {
|
||||||
|
builder = addCreateHiveTableArgs(builder, getHiveExternalTableName());
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,190 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static junit.framework.TestCase.fail;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.TextFileTestUtils;
|
||||||
|
import org.apache.sqoop.util.password.CredentialProviderHelper;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.contrib.java.lang.system.EnvironmentVariables;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.file.Files;
|
||||||
|
|
||||||
|
public abstract class AbstractTestImportWithHadoopCredProvider extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestImportWithHadoopCredProvider.class.getName());
|
||||||
|
|
||||||
|
private static String providerPathDefault;
|
||||||
|
private static String providerPathEnv;
|
||||||
|
private static String providerPathPwdFile;
|
||||||
|
|
||||||
|
protected static CloudCredentialsRule credentialsRule;
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static final EnvironmentVariables environmentVariables = new EnvironmentVariables();
|
||||||
|
private static File providerFileDefault;
|
||||||
|
private static File providerFileEnvPwd;
|
||||||
|
private static File providerFilePwdFile;
|
||||||
|
|
||||||
|
private static final String PASSWORD_FILE_NAME = "password-file.txt";
|
||||||
|
private static final String HADOOP_CREDSTORE_PASSWORD_ENV_NAME = "HADOOP_CREDSTORE_PASSWORD";
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
private final String credentialProviderPathProperty;
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void fillCredentialProviderFiles() throws Exception {
|
||||||
|
generateTempProviderFileNames();
|
||||||
|
fillCredentialProviderDefault();
|
||||||
|
fillCredentialProviderPwdFile();
|
||||||
|
fillCredentialProviderEnv();
|
||||||
|
}
|
||||||
|
|
||||||
|
public AbstractTestImportWithHadoopCredProvider() {
|
||||||
|
this(CredentialProviderHelper.HADOOP_CREDENTIAL_PROVIDER_PATH);
|
||||||
|
}
|
||||||
|
|
||||||
|
public AbstractTestImportWithHadoopCredProvider(String credentialProviderPathProperty) {
|
||||||
|
super(credentialsRule);
|
||||||
|
this.credentialProviderPathProperty = credentialProviderPathProperty;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() {
|
||||||
|
super.setUp();
|
||||||
|
environmentVariables.clear(HADOOP_CREDSTORE_PASSWORD_ENV_NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void deleteTemporaryCredFiles() {
|
||||||
|
deleteFileOnExit(providerFileDefault);
|
||||||
|
deleteFileOnExit(providerFileEnvPwd);
|
||||||
|
deleteFileOnExit(providerFilePwdFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderDefaultSucceeds() throws Exception {
|
||||||
|
runImport(getArgs(providerPathDefault, false, null));
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderEnvSucceeds() throws Exception {
|
||||||
|
setHadoopCredStorePwdEnvVar();
|
||||||
|
runImport(getArgs(providerPathEnv, false, null));
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderPwdFileSucceeds() throws Exception {
|
||||||
|
runImport(getArgs(providerPathPwdFile, true, PASSWORD_FILE_NAME));
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderWithNoProviderPathFails() throws Exception {
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(getArgs(null, false, null));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderWithNoEnvFails() throws Exception {
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(getArgs(providerPathEnv, false, null));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderWithWrongPwdFileFails() throws Exception {
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(getArgs(providerPathPwdFile, true, "wrong-password-file.txt"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCredentialProviderWithNoPwdFileFails() throws Exception {
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(getArgs(providerPathPwdFile, true, null));
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgs(String providerPath, boolean withPwdFile, String pwdFile) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForHadoopCredProviderUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
|
||||||
|
builder.withProperty(credentialProviderPathProperty, providerPath);
|
||||||
|
if (withPwdFile) {
|
||||||
|
builder.withProperty(CredentialProviderHelper.CREDENTIAL_PROVIDER_PASSWORD_FILE, pwdFile);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void fillCredentialProviderDefault() throws Exception {
|
||||||
|
credentialsRule.fillCredentialProvider(new Configuration(), providerPathDefault);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void fillCredentialProviderEnv() throws Exception {
|
||||||
|
setHadoopCredStorePwdEnvVar();
|
||||||
|
credentialsRule.fillCredentialProvider(new Configuration(), providerPathEnv);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void fillCredentialProviderPwdFile() throws Exception {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.set(CredentialProviderHelper.CREDENTIAL_PROVIDER_PASSWORD_FILE, PASSWORD_FILE_NAME);
|
||||||
|
credentialsRule.fillCredentialProvider(conf, providerPathPwdFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void generateTempProviderFileNames() throws IOException {
|
||||||
|
providerFileDefault = Files.createTempFile("test-default-pwd-", ".jceks").toFile();
|
||||||
|
boolean deleted = providerFileDefault.delete();
|
||||||
|
providerFileEnvPwd = Files.createTempFile("test-env-pwd-", ".jceks").toFile();
|
||||||
|
deleted &= providerFileEnvPwd.delete();
|
||||||
|
providerFilePwdFile = Files.createTempFile("test-file-pwd-", ".jceks").toFile();
|
||||||
|
deleted &= providerFilePwdFile.delete();
|
||||||
|
if (!deleted) {
|
||||||
|
fail("Could not delete temporary provider files");
|
||||||
|
}
|
||||||
|
providerPathDefault = "jceks://file/" + providerFileDefault.getAbsolutePath();
|
||||||
|
providerPathEnv = "jceks://file/" + providerFileEnvPwd.getAbsolutePath();
|
||||||
|
providerPathPwdFile = "jceks://file/" + providerFilePwdFile.getAbsolutePath();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void setHadoopCredStorePwdEnvVar() {
|
||||||
|
environmentVariables.set(HADOOP_CREDSTORE_PASSWORD_ENV_NAME, "credProviderPwd");
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void deleteFileOnExit(File file) {
|
||||||
|
if (file != null) {
|
||||||
|
file.deleteOnExit();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,98 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.AvroTestUtils;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestIncrementalAppendAvroImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestIncrementalAppendAvroImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestIncrementalAppendAvroImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsAvroDataFileWhenNoNewRowIsImported() throws IOException {
|
||||||
|
String[] args = getArgsWithAsAvroDataFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsAvroDataFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
failIfOutputFilePathContainingPatternExists(fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsAvroDataFile() throws IOException {
|
||||||
|
String[] args = getArgsWithAsAvroDataFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsAvroDataFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
AvroTestUtils.verify(getDataSet().getExpectedExtraAvroOutput(), fileSystemRule.getCloudFileSystem().getConf(), fileSystemRule.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsAvroDataFileWithMapreduceOutputBasenameProperty() throws IOException {
|
||||||
|
String[] args = getArgsWithAsAvroDataFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsAvroDataFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
AvroTestUtils.verify(getDataSet().getExpectedExtraAvroOutput(), fileSystemRule.getCloudFileSystem().getConf(), fileSystemRule.getTargetDirPath(), CUSTOM_MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsAvroDataFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-avrodatafile");
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getIncrementalAppendArgsWithAsAvroDataFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-avrodatafile");
|
||||||
|
builder = addIncrementalAppendImportArgs(builder, fileSystemRule.getTemporaryRootDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,104 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.util.ParquetReader;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class AbstractTestIncrementalAppendParquetImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestIncrementalAppendParquetImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestIncrementalAppendParquetImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsParquetFileWhenNoNewRowIsImported() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutput(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsParquetFile() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutputAfterAppend(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsParquetFileWithMapreduceOutputBasenameProperty() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsParquetFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
failIfOutputFilePathContainingPatternDoesNotExists(fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutputAfterAppend(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-parquetfile");
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getIncrementalAppendArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-parquetfile");
|
||||||
|
builder = addIncrementalAppendImportArgs(builder, fileSystemRule.getTemporaryRootDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,96 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.SequenceFileTestUtils;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestIncrementalAppendSequenceFileImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestIncrementalAppendSequenceFileImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestIncrementalAppendSequenceFileImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsSequenceFileWhenNoNewRowIsImported() throws Exception {
|
||||||
|
String[] args = getArgsWithAsSequenceFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsSequenceFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
failIfOutputFilePathContainingPatternExists(fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsSequenceFile() throws Exception {
|
||||||
|
String[] args = getArgsWithAsSequenceFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsSequenceFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
SequenceFileTestUtils.verify(this, getDataSet().getExpectedExtraSequenceFileOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsSequenceFileWithMapreduceOutputBasenameProperty() throws Exception {
|
||||||
|
String[] args = getArgsWithAsSequenceFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgsWithAsSequenceFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
SequenceFileTestUtils.verify(this, getDataSet().getExpectedExtraSequenceFileOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), CUSTOM_MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsSequenceFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-sequencefile");
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getIncrementalAppendArgsWithAsSequenceFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-sequencefile");
|
||||||
|
builder = addIncrementalAppendImportArgs(builder, fileSystemRule.getTemporaryRootDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,98 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.TextFileTestUtils;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestIncrementalAppendTextImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestIncrementalAppendTextImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestIncrementalAppendTextImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsTextFileWhenNoNewRowIsImported() throws IOException {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
failIfOutputFilePathContainingPatternExists(fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsTextFile() throws IOException {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedExtraTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalAppendAsTextFileWithMapreduceOutputBasenameProperty() throws IOException {
|
||||||
|
String[] args = getArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
insertInputDataIntoTable(getDataSet().getExtraInputData());
|
||||||
|
|
||||||
|
args = getIncrementalAppendArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedExtraTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), CUSTOM_MAP_OUTPUT_FILE_00001);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgs(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getIncrementalAppendArgs(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
builder = addIncrementalAppendImportArgs(builder, fileSystemRule.getTemporaryRootDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,118 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.util.ParquetReader;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class AbstractTestIncrementalMergeParquetImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestIncrementalMergeParquetImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
super.setUp();
|
||||||
|
createTestTableFromInitialInputDataForMerge();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected AbstractTestIncrementalMergeParquetImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalMergeAsParquetFileWhenNoNewRowIsImported() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
clearTable(getTableName());
|
||||||
|
|
||||||
|
args = getIncrementalMergeArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutputWithTimestampColumn(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalMergeAsParquetFile() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
clearTable(getTableName());
|
||||||
|
|
||||||
|
insertInputDataIntoTableForMerge(getDataSet().getNewInputDataForMerge());
|
||||||
|
|
||||||
|
args = getIncrementalMergeArgsWithAsParquetFileOption(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutputWithTimestampColumnAfterMerge(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalMergeAsParquetFileWithMapreduceOutputBasenameProperty() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
clearTable(getTableName());
|
||||||
|
|
||||||
|
insertInputDataIntoTableForMerge(getDataSet().getNewInputDataForMerge());
|
||||||
|
|
||||||
|
args = getIncrementalMergeArgsWithAsParquetFileOption(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
failIfOutputFilePathContainingPatternDoesNotExists(fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutputWithTimestampColumnAfterMerge(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-parquetfile");
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getIncrementalMergeArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-parquetfile");
|
||||||
|
builder = addIncrementalMergeImportArgs(builder, fileSystemRule.getTemporaryRootDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,110 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.TextFileTestUtils;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestIncrementalMergeTextImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestIncrementalMergeTextImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
super.setUp();
|
||||||
|
createTestTableFromInitialInputDataForMerge();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected AbstractTestIncrementalMergeTextImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalMergeAsTextFileWhenNoNewRowIsImported() throws Exception {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
clearTable(getTableName());
|
||||||
|
|
||||||
|
args = getIncrementalMergeArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutputBeforeMerge(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), REDUCE_OUTPUT_FILE_00000);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalMergeAsTextFile() throws Exception {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
clearTable(getTableName());
|
||||||
|
|
||||||
|
insertInputDataIntoTableForMerge(getDataSet().getNewInputDataForMerge());
|
||||||
|
|
||||||
|
args = getIncrementalMergeArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutputAfterMerge(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), REDUCE_OUTPUT_FILE_00000);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIncrementalMergeAsTextFileWithMapreduceOutputBasenameProperty() throws Exception {
|
||||||
|
String[] args = getArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
clearTable(getTableName());
|
||||||
|
|
||||||
|
insertInputDataIntoTableForMerge(getDataSet().getNewInputDataForMerge());
|
||||||
|
|
||||||
|
args = getIncrementalMergeArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutputAfterMerge(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath(), CUSTOM_REDUCE_OUTPUT_FILE_00000);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgs(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getIncrementalMergeArgs(boolean withMapreduceOutputBasenameProperty) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
builder = addIncrementalMergeImportArgs(builder, fileSystemRule.getTemporaryRootDirPath().toString());
|
||||||
|
if (withMapreduceOutputBasenameProperty) {
|
||||||
|
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, MAPREDUCE_OUTPUT_BASENAME);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,85 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.util.ParquetReader;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class AbstractTestParquetImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestParquetImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestParquetImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsParquetFileWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutput(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsParquetFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getArgsWithAsParquetFileAndDeleteTargetDirOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
List<String> result = new ParquetReader(fileSystemRule.getTargetDirPath(), fileSystemRule.getCloudFileSystem().getConf()).readAllInCsvSorted();
|
||||||
|
assertEquals(getDataSet().getExpectedParquetOutput(), result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsParquetFileWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
||||||
|
String[] args = getArgsWithAsParquetFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsParquetFileOption() {
|
||||||
|
return getArgsForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-parquetfile");
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsParquetFileAndDeleteTargetDirOption() {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-parquetfile");
|
||||||
|
builder.withOption("delete-target-dir");
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,78 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.SequenceFileTestUtils;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestSequenceFileImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestSequenceFileImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestSequenceFileImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsSequenceFileWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws Exception {
|
||||||
|
String[] args = getArgsWithAsSequenceFileOption();
|
||||||
|
runImport(args);
|
||||||
|
SequenceFileTestUtils.verify(this, getDataSet().getExpectedSequenceFileOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsSequenceFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
||||||
|
String[] args = getArgsWithAsSequenceFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getArgsWithAsSequenceFileAndDeleteTargetDirOption();
|
||||||
|
runImport(args);
|
||||||
|
SequenceFileTestUtils.verify(this, getDataSet().getExpectedSequenceFileOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsSequenceWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
||||||
|
String[] args = getArgsWithAsSequenceFileOption();
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsSequenceFileOption() {
|
||||||
|
return getArgsForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-sequencefile");
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithAsSequenceFileAndDeleteTargetDirOption() {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTestsWithFileFormatOption(fileSystemRule.getTargetDirPath().toString(), "as-sequencefile");
|
||||||
|
builder.withOption("delete-target-dir");
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
111
src/test/org/apache/sqoop/cloud/AbstractTestTextImport.java
Normal file
111
src/test/org/apache/sqoop/cloud/AbstractTestTextImport.java
Normal file
@ -0,0 +1,111 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.TextFileTestUtils;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public abstract class AbstractTestTextImport extends CloudImportJobTestCase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(AbstractTestTextImport.class.getName());
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
protected AbstractTestTextImport(CloudCredentialsRule credentialsRule) {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws IOException {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getArgsWithDeleteTargetOption(false);
|
||||||
|
runImport(args);
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
||||||
|
String[] args = getArgs(false);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsTextFile() throws IOException {
|
||||||
|
String[] args = getArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsTextFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
||||||
|
String[] args = getArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
args = getArgsWithDeleteTargetOption(true);
|
||||||
|
runImport(args);
|
||||||
|
TextFileTestUtils.verify(getDataSet().getExpectedTextOutput(), fileSystemRule.getCloudFileSystem(), fileSystemRule.getTargetDirPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportAsTextFileWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
||||||
|
String[] args = getArgs(true);
|
||||||
|
runImport(args);
|
||||||
|
|
||||||
|
thrown.expect(IOException.class);
|
||||||
|
runImport(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgs(boolean withAsTextFileOption) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
if (withAsTextFileOption) {
|
||||||
|
builder.withOption("as-textfile");
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String[] getArgsWithDeleteTargetOption(boolean withAsTextFileOption) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(fileSystemRule.getTargetDirPath().toString());
|
||||||
|
builder.withOption("delete-target-dir");
|
||||||
|
if (withAsTextFileOption) {
|
||||||
|
builder.withOption("as-textfile");
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
208
src/test/org/apache/sqoop/cloud/CloudImportJobTestCase.java
Normal file
208
src/test/org/apache/sqoop/cloud/CloudImportJobTestCase.java
Normal file
@ -0,0 +1,208 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud;
|
||||||
|
|
||||||
|
import static org.apache.sqoop.tool.BaseSqoopTool.FMT_PARQUETFILE_ARG;
|
||||||
|
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudFileSystemRule;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudTestDataSet;
|
||||||
|
import org.apache.sqoop.hive.minicluster.HiveMiniCluster;
|
||||||
|
import org.apache.sqoop.hive.minicluster.NoAuthenticationConfiguration;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.apache.sqoop.testutil.ImportJobTestCase;
|
||||||
|
import org.apache.sqoop.util.FileSystemUtil;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class CloudImportJobTestCase extends ImportJobTestCase {
|
||||||
|
|
||||||
|
public static final String MAPREDUCE_OUTPUT_BASENAME = "custom";
|
||||||
|
public static final String CUSTOM_MAP_OUTPUT_FILE_00001 = MAPREDUCE_OUTPUT_BASENAME + "-m-00001";
|
||||||
|
public static final String CUSTOM_REDUCE_OUTPUT_FILE_00000 = MAPREDUCE_OUTPUT_BASENAME + "-r-00000";
|
||||||
|
|
||||||
|
private static final String HIVE_EXTERNAL_TABLE_NAME = "test_external_table";
|
||||||
|
|
||||||
|
private final CloudTestDataSet dataSet;
|
||||||
|
|
||||||
|
private final CloudCredentialsRule credentialsRule;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public CloudFileSystemRule fileSystemRule;
|
||||||
|
|
||||||
|
protected CloudImportJobTestCase(CloudCredentialsRule credentialsRule) {
|
||||||
|
this.credentialsRule = credentialsRule;
|
||||||
|
this.fileSystemRule = new CloudFileSystemRule(credentialsRule);
|
||||||
|
this.dataSet = new CloudTestDataSet();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
super.setUp();
|
||||||
|
createTestTableFromInputData();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@After
|
||||||
|
public void tearDown() {
|
||||||
|
super.tearDown();
|
||||||
|
System.clearProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder getArgumentArrayBuilderForUnitTests(String targetDir) {
|
||||||
|
ArgumentArrayBuilder builder = new ArgumentArrayBuilder();
|
||||||
|
credentialsRule.addCloudCredentialProperties(builder);
|
||||||
|
return builder.withCommonHadoopFlags()
|
||||||
|
.withOption("connect", getConnectString())
|
||||||
|
.withOption("num-mappers", "1")
|
||||||
|
.withOption("table", getTableName())
|
||||||
|
.withOption("target-dir", targetDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder getArgumentArrayBuilderForHadoopCredProviderUnitTests(String targetDir) {
|
||||||
|
|
||||||
|
ArgumentArrayBuilder builder = new ArgumentArrayBuilder();
|
||||||
|
credentialsRule.addCloudCredentialProviderProperties(builder);
|
||||||
|
return builder.withCommonHadoopFlags()
|
||||||
|
.withOption("connect", getConnectString())
|
||||||
|
.withOption("num-mappers", "1")
|
||||||
|
.withOption("table", getTableName())
|
||||||
|
.withOption("target-dir", targetDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder getArgumentArrayBuilderForUnitTestsWithFileFormatOption(String targetDir,
|
||||||
|
String fileFormat) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(targetDir);
|
||||||
|
builder.withOption(fileFormat);
|
||||||
|
useParquetHadoopAPIImplementationIfAsParquet(builder, fileFormat);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String[] getArgsForUnitTestsWithFileFormatOption(String targetDir,
|
||||||
|
String fileFormat) {
|
||||||
|
ArgumentArrayBuilder builder = getArgumentArrayBuilderForUnitTests(targetDir);
|
||||||
|
builder.withOption(fileFormat);
|
||||||
|
useParquetHadoopAPIImplementationIfAsParquet(builder, fileFormat);
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void useParquetHadoopAPIImplementationIfAsParquet(ArgumentArrayBuilder builder, String fileFormat) {
|
||||||
|
// Parquet import to Cloud storages is supported only with the Parquet Hadoop API implementation.
|
||||||
|
if (fileFormat.equals(FMT_PARQUETFILE_ARG)) {
|
||||||
|
builder.withOption("parquet-configurator-implementation", "hadoop");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder addExternalHiveTableImportArgs(ArgumentArrayBuilder builder,
|
||||||
|
String hs2Url,
|
||||||
|
String externalTableDir) {
|
||||||
|
return builder
|
||||||
|
.withOption("hive-import")
|
||||||
|
.withOption("hs2-url", hs2Url)
|
||||||
|
.withOption("external-table-dir", externalTableDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder addCreateHiveTableArgs(ArgumentArrayBuilder builder, String hiveExternalTableDirName) {
|
||||||
|
return builder
|
||||||
|
.withOption("create-hive-table")
|
||||||
|
.withOption("hive-table", hiveExternalTableDirName);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder addIncrementalAppendImportArgs(ArgumentArrayBuilder builder, String temporaryRootDir) {
|
||||||
|
return builder
|
||||||
|
.withOption("incremental", "append")
|
||||||
|
.withOption("check-column", "ID")
|
||||||
|
.withOption("last-value", "4")
|
||||||
|
.withOption("temporary-rootdir", temporaryRootDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ArgumentArrayBuilder addIncrementalMergeImportArgs(ArgumentArrayBuilder builder, String temporaryRootDir) {
|
||||||
|
return builder
|
||||||
|
.withOption("incremental", "lastmodified")
|
||||||
|
.withOption("check-column", "RECORD_DATE")
|
||||||
|
.withOption("merge-key", "DEBUT")
|
||||||
|
.withOption("last-value", dataSet.getInitialTimestampForMerge())
|
||||||
|
.withOption("temporary-rootdir", temporaryRootDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void createTestTableFromInputData() {
|
||||||
|
List<String[]> inputData = dataSet.getInputData();
|
||||||
|
createTableWithColTypesAndNames(dataSet.getColumnNames(), dataSet.getColumnTypes(), new String[0]);
|
||||||
|
for (String[] dataRow : inputData) {
|
||||||
|
insertInputDataIntoTable(dataRow);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void insertInputDataIntoTable(String[] inputData) {
|
||||||
|
insertIntoTable(dataSet.getColumnNames(), dataSet.getColumnTypes(), inputData);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void createTestTableFromInitialInputDataForMerge() {
|
||||||
|
createTableWithRecordsWithColTypesAndNames(dataSet.getColumnNamesForMerge(), dataSet.getColumnTypesForMerge(), dataSet.getInitialInputDataForMerge());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void insertInputDataIntoTableForMerge(List<List<Object>> inputData) {
|
||||||
|
insertRecordsIntoTableWithColTypesAndNames(dataSet.getColumnNamesForMerge(), dataSet.getColumnTypesForMerge(), inputData);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void failIfOutputFilePathContainingPatternExists(FileSystem cloudFileSystem, Path targetDirPath, String pattern) throws IOException {
|
||||||
|
List<Path> outputFilesWithPathContainingPattern = FileSystemUtil.findFilesWithPathContainingPattern(targetDirPath,
|
||||||
|
cloudFileSystem.getConf(), pattern);
|
||||||
|
if (outputFilesWithPathContainingPattern.size() != 0) {
|
||||||
|
fail("No output file was expected with pattern" + pattern);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void failIfOutputFilePathContainingPatternDoesNotExists(FileSystem cloudFileSystem, Path targetDirPath, String pattern) throws IOException {
|
||||||
|
List<Path> outputFilesWithPathContainingPattern = FileSystemUtil.findFilesWithPathContainingPattern(targetDirPath,
|
||||||
|
cloudFileSystem.getConf(), pattern);
|
||||||
|
if (outputFilesWithPathContainingPattern.size() == 0) {
|
||||||
|
fail("No output file was found with pattern" + pattern);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected CloudTestDataSet getDataSet() {
|
||||||
|
return dataSet;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HiveMiniCluster createCloudBasedHiveMiniCluster() {
|
||||||
|
HiveMiniCluster hiveMiniCluster = new HiveMiniCluster(new NoAuthenticationConfiguration());
|
||||||
|
hiveMiniCluster.start();
|
||||||
|
credentialsRule.addCloudCredentialProperties(hiveMiniCluster.getConfig());
|
||||||
|
|
||||||
|
return hiveMiniCluster;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getHiveExternalTableName() {
|
||||||
|
return HIVE_EXTERNAL_TABLE_NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected CloudCredentialsRule getCredentialsRule() {
|
||||||
|
return credentialsRule;
|
||||||
|
}
|
||||||
|
}
|
98
src/test/org/apache/sqoop/cloud/s3/S3CredentialsRule.java
Normal file
98
src/test/org/apache/sqoop/cloud/s3/S3CredentialsRule.java
Normal file
@ -0,0 +1,98 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Constants.SESSION_TOKEN;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.s3a.Constants;
|
||||||
|
import org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider;
|
||||||
|
import org.apache.sqoop.cloud.tools.CloudCredentialsRule;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
public class S3CredentialsRule extends CloudCredentialsRule {
|
||||||
|
|
||||||
|
private static final String PROPERTY_GENERATOR_COMMAND = "s3.generator.command";
|
||||||
|
|
||||||
|
private static final String PROPERTY_BUCKET_URL = "s3.bucket.url";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addCloudCredentialProperties(Configuration hadoopConf) {
|
||||||
|
hadoopConf.set(ACCESS_KEY, credentialsMap.get(ACCESS_KEY));
|
||||||
|
hadoopConf.set(SECRET_KEY, credentialsMap.get(SECRET_KEY));
|
||||||
|
|
||||||
|
if (credentialsMap.containsKey(SESSION_TOKEN)) {
|
||||||
|
hadoopConf.set(SESSION_TOKEN, credentialsMap.get(SESSION_TOKEN));
|
||||||
|
hadoopConf.set(Constants.AWS_CREDENTIALS_PROVIDER, TemporaryAWSCredentialsProvider.class.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Default filesystem needs to be set to S3 for the output verification phase
|
||||||
|
hadoopConf.set("fs.defaultFS", getBaseCloudDirectoryUrl());
|
||||||
|
|
||||||
|
// FileSystem has a static cache that should be disabled during tests to make sure
|
||||||
|
// Sqoop relies on the S3 credentials set via the -D system properties.
|
||||||
|
// For details please see SQOOP-3383
|
||||||
|
hadoopConf.setBoolean("fs.s3a.impl.disable.cache", true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addCloudCredentialProperties(ArgumentArrayBuilder builder) {
|
||||||
|
builder
|
||||||
|
.withProperty(Constants.ACCESS_KEY, credentialsMap.get(ACCESS_KEY))
|
||||||
|
.withProperty(Constants.SECRET_KEY, credentialsMap.get(SECRET_KEY));
|
||||||
|
if (credentialsMap.containsKey(SESSION_TOKEN)) {
|
||||||
|
builder.withProperty(Constants.SESSION_TOKEN, credentialsMap.get(SESSION_TOKEN))
|
||||||
|
.withProperty(Constants.AWS_CREDENTIALS_PROVIDER, TemporaryAWSCredentialsProvider.class.getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addCloudCredentialProviderProperties(ArgumentArrayBuilder builder) {
|
||||||
|
builder.withProperty("fs.s3a.impl.disable.cache", "true");
|
||||||
|
if (credentialsMap.containsKey(SESSION_TOKEN)) {
|
||||||
|
builder.withProperty(Constants.AWS_CREDENTIALS_PROVIDER, TemporaryAWSCredentialsProvider.class.getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getBaseCloudDirectoryUrl() {
|
||||||
|
String propertyBucketUrl = System.getProperty(PROPERTY_BUCKET_URL);
|
||||||
|
return propertyBucketUrl.concat("/");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void initializeCredentialsMap(Iterable<String> credentials) {
|
||||||
|
Iterator<String> credentialsIterator = credentials.iterator();
|
||||||
|
|
||||||
|
credentialsMap.put(ACCESS_KEY, credentialsIterator.next());
|
||||||
|
credentialsMap.put(SECRET_KEY, credentialsIterator.next());
|
||||||
|
if (credentialsIterator.hasNext()) {
|
||||||
|
credentialsMap.put(SESSION_TOKEN, credentialsIterator.next());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getGeneratorCommand() {
|
||||||
|
return System.getProperty(PROPERTY_GENERATOR_COMMAND);
|
||||||
|
}
|
||||||
|
}
|
36
src/test/org/apache/sqoop/cloud/s3/TestS3AvroImport.java
Normal file
36
src/test/org/apache/sqoop/cloud/s3/TestS3AvroImport.java
Normal file
@ -0,0 +1,36 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestAvroImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3AvroImport extends AbstractTestAvroImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3AvroImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,42 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestExternalHiveTableImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.apache.sqoop.util.BlockJUnit4ClassRunnerWithParametersFactory;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
@Parameterized.UseParametersRunnerFactory(BlockJUnit4ClassRunnerWithParametersFactory.class)
|
||||||
|
public class TestS3ExternalHiveTableImport extends AbstractTestExternalHiveTableImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3ExternalHiveTableImport(String fileFormatArg, List<String> expectedResult) {
|
||||||
|
super(credentialsRule, fileFormatArg, expectedResult);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,53 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestImportWithHadoopCredProvider;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.apache.sqoop.util.BlockJUnit4ClassRunnerWithParametersFactory;
|
||||||
|
import org.apache.sqoop.util.password.CredentialProviderHelper;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
@Parameterized.UseParametersRunnerFactory(BlockJUnit4ClassRunnerWithParametersFactory.class)
|
||||||
|
public class TestS3ImportWithHadoopCredProvider extends AbstractTestImportWithHadoopCredProvider {
|
||||||
|
|
||||||
|
@Parameterized.Parameters(name = "credentialProviderPathProperty = {0}")
|
||||||
|
public static Iterable<? extends Object> parameters() {
|
||||||
|
return Arrays.asList(CredentialProviderHelper.HADOOP_CREDENTIAL_PROVIDER_PATH,
|
||||||
|
CredentialProviderHelper.S3A_CREDENTIAL_PROVIDER_PATH);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule s3CredentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
static {
|
||||||
|
AbstractTestImportWithHadoopCredProvider.credentialsRule = s3CredentialsRule;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TestS3ImportWithHadoopCredProvider(String credentialProviderPathProperty) {
|
||||||
|
super(credentialProviderPathProperty);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestIncrementalAppendAvroImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3IncrementalAppendAvroImport extends AbstractTestIncrementalAppendAvroImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3IncrementalAppendAvroImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestIncrementalAppendParquetImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3IncrementalAppendParquetImport extends AbstractTestIncrementalAppendParquetImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3IncrementalAppendParquetImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestIncrementalAppendSequenceFileImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3IncrementalAppendSequenceFileImport extends AbstractTestIncrementalAppendSequenceFileImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3IncrementalAppendSequenceFileImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestIncrementalAppendTextImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3IncrementalAppendTextImport extends AbstractTestIncrementalAppendTextImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3IncrementalAppendTextImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestIncrementalMergeParquetImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3IncrementalMergeParquetImport extends AbstractTestIncrementalMergeParquetImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3IncrementalMergeParquetImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestIncrementalMergeTextImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3IncrementalMergeTextImport extends AbstractTestIncrementalMergeTextImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3IncrementalMergeTextImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
35
src/test/org/apache/sqoop/cloud/s3/TestS3ParquetImport.java
Normal file
35
src/test/org/apache/sqoop/cloud/s3/TestS3ParquetImport.java
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestParquetImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3ParquetImport extends AbstractTestParquetImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3ParquetImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestSequenceFileImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3SequenceFileImport extends AbstractTestSequenceFileImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3SequenceFileImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
35
src/test/org/apache/sqoop/cloud/s3/TestS3TextImport.java
Normal file
35
src/test/org/apache/sqoop/cloud/s3/TestS3TextImport.java
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.s3;
|
||||||
|
|
||||||
|
import org.apache.sqoop.cloud.AbstractTestTextImport;
|
||||||
|
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(S3Test.class)
|
||||||
|
public class TestS3TextImport extends AbstractTestTextImport {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static S3CredentialsRule credentialsRule = new S3CredentialsRule();
|
||||||
|
|
||||||
|
public TestS3TextImport() {
|
||||||
|
super(credentialsRule);
|
||||||
|
}
|
||||||
|
}
|
111
src/test/org/apache/sqoop/cloud/tools/CloudCredentialsRule.java
Normal file
111
src/test/org/apache/sqoop/cloud/tools/CloudCredentialsRule.java
Normal file
@ -0,0 +1,111 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.tools;
|
||||||
|
|
||||||
|
import static org.apache.commons.lang3.StringUtils.isNotBlank;
|
||||||
|
import static org.junit.Assume.assumeTrue;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.security.alias.CredentialShell;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
||||||
|
import org.junit.rules.TestRule;
|
||||||
|
import org.junit.runner.Description;
|
||||||
|
import org.junit.runners.model.Statement;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The task of this rule is to execute the credential generator command for the given cloud storage
|
||||||
|
* and store its output for later use.
|
||||||
|
* If the credential generator command is not provided the tests using this rule will be skipped.
|
||||||
|
*
|
||||||
|
* The subclasses of this rule have to define:
|
||||||
|
* <ul>
|
||||||
|
* <li>what is the credential generator command</li>
|
||||||
|
* <li>how to use its output to create the credentials map</li>
|
||||||
|
* <li>how apply the credentials to the Hadoop configurations and Sqoop arguments for the tests</li>
|
||||||
|
* </ul>
|
||||||
|
*/
|
||||||
|
public abstract class CloudCredentialsRule implements TestRule {
|
||||||
|
|
||||||
|
protected final Map<String, String> credentialsMap;
|
||||||
|
|
||||||
|
private final CredentialGenerator credentialGenerator;
|
||||||
|
|
||||||
|
protected CloudCredentialsRule() {
|
||||||
|
this(new CredentialGenerator());
|
||||||
|
}
|
||||||
|
|
||||||
|
public CloudCredentialsRule(CredentialGenerator credentialGenerator) {
|
||||||
|
this.credentialGenerator = credentialGenerator;
|
||||||
|
this.credentialsMap = new HashMap<>();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Statement apply(Statement base, Description description) {
|
||||||
|
return new Statement() {
|
||||||
|
@Override
|
||||||
|
public void evaluate() throws Throwable {
|
||||||
|
assumeTrue(isNotBlank(getGeneratorCommand()));
|
||||||
|
|
||||||
|
setupCredentials();
|
||||||
|
base.evaluate();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public void fillCredentialProvider(Configuration conf, String providerPath) {
|
||||||
|
credentialsMap.forEach(
|
||||||
|
(key, value) -> runCredentialProviderCreateCommand(getCreateCommand(key, value, providerPath), conf)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setupCredentials() throws IOException {
|
||||||
|
Iterable<String> credentials = credentialGenerator.invokeGeneratorCommand(getGeneratorCommand());
|
||||||
|
|
||||||
|
initializeCredentialsMap(credentials);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void runCredentialProviderCreateCommand(String command, Configuration conf) {
|
||||||
|
try {
|
||||||
|
ToolRunner.run(conf, new CredentialShell(), command.split(" "));
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getCreateCommand(String credentialKey, String credentialValue, String providerPath) {
|
||||||
|
return "create " + credentialKey + " -value " + credentialValue + " -provider " + providerPath;
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract void addCloudCredentialProperties(Configuration hadoopConf);
|
||||||
|
|
||||||
|
public abstract void addCloudCredentialProperties(ArgumentArrayBuilder builder);
|
||||||
|
|
||||||
|
public abstract void addCloudCredentialProviderProperties(ArgumentArrayBuilder builder);
|
||||||
|
|
||||||
|
public abstract String getBaseCloudDirectoryUrl();
|
||||||
|
|
||||||
|
protected abstract void initializeCredentialsMap(Iterable<String> credentials);
|
||||||
|
|
||||||
|
protected abstract String getGeneratorCommand();
|
||||||
|
}
|
135
src/test/org/apache/sqoop/cloud/tools/CloudFileSystemRule.java
Normal file
135
src/test/org/apache/sqoop/cloud/tools/CloudFileSystemRule.java
Normal file
@ -0,0 +1,135 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.tools;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.junit.rules.TestRule;
|
||||||
|
import org.junit.runner.Description;
|
||||||
|
import org.junit.runners.model.Statement;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The task of this rule is to initialize the cloud FileSystem object and the test directories
|
||||||
|
* before the tests and clean them up after.
|
||||||
|
* This FileSystem object is used to assert the output of a Sqoop job.
|
||||||
|
* If the credentials are not provided for the cloud system the tests will be skipped.
|
||||||
|
*/
|
||||||
|
public class CloudFileSystemRule implements TestRule {
|
||||||
|
|
||||||
|
private static final Log LOG = LogFactory.getLog(CloudFileSystemRule.class.getName());
|
||||||
|
|
||||||
|
private static final String TARGET_DIR_NAME_PREFIX = "testdir";
|
||||||
|
|
||||||
|
private static final String HIVE_EXTERNAL_DIR_NAME_PREFIX = "externaldir";
|
||||||
|
|
||||||
|
private static final String TEMP_DIR = "tmp";
|
||||||
|
|
||||||
|
private static final String TEMPORARY_ROOTDIR_SUFFIX = "_temprootdir";
|
||||||
|
|
||||||
|
private final CloudCredentialsRule credentialsRule;
|
||||||
|
|
||||||
|
private FileSystem fileSystem;
|
||||||
|
|
||||||
|
private String targetDirName;
|
||||||
|
|
||||||
|
private String hiveExternalTableDirName;
|
||||||
|
|
||||||
|
public CloudFileSystemRule(CloudCredentialsRule credentialsRule) {
|
||||||
|
this.credentialsRule = credentialsRule;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Statement apply(Statement base, Description description) {
|
||||||
|
return new Statement() {
|
||||||
|
@Override
|
||||||
|
public void evaluate() throws Throwable {
|
||||||
|
setUp();
|
||||||
|
try {
|
||||||
|
base.evaluate();
|
||||||
|
} finally {
|
||||||
|
tearDown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public FileSystem getCloudFileSystem() {
|
||||||
|
return fileSystem;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Path getTargetDirPath() {
|
||||||
|
return new Path(getCloudTempDirPath(), targetDirName);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Path getTemporaryRootDirPath() {
|
||||||
|
return new Path(getTargetDirPath().toString() + TEMPORARY_ROOTDIR_SUFFIX);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Path getExternalTableDirPath() {
|
||||||
|
return new Path(getCloudTempDirPath(), hiveExternalTableDirName);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setUp() throws IOException {
|
||||||
|
initializeCloudFileSystem();
|
||||||
|
initializeTestDirectoryNames();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initializeCloudFileSystem() throws IOException {
|
||||||
|
Configuration hadoopConf = new Configuration();
|
||||||
|
credentialsRule.addCloudCredentialProperties(hadoopConf);
|
||||||
|
|
||||||
|
fileSystem = FileSystem.get(hadoopConf);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initializeTestDirectoryNames() {
|
||||||
|
targetDirName = generateUniqueDirName(TARGET_DIR_NAME_PREFIX);
|
||||||
|
hiveExternalTableDirName = generateUniqueDirName(HIVE_EXTERNAL_DIR_NAME_PREFIX);
|
||||||
|
}
|
||||||
|
|
||||||
|
private String generateUniqueDirName(String dirPrefix) {
|
||||||
|
String uuid = UUID.randomUUID().toString();
|
||||||
|
return dirPrefix + "-" + uuid;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void cleanUpDirectory(Path directoryPath) {
|
||||||
|
try {
|
||||||
|
if (fileSystem.exists(directoryPath)) {
|
||||||
|
fileSystem.delete(directoryPath, true);
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Issue with cleaning up directory", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Path getCloudTempDirPath() {
|
||||||
|
return new Path(credentialsRule.getBaseCloudDirectoryUrl(), TEMP_DIR);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void tearDown() {
|
||||||
|
cleanUpDirectory(getTemporaryRootDirPath());
|
||||||
|
cleanUpDirectory(getTargetDirPath());
|
||||||
|
cleanUpDirectory(getExternalTableDirPath());
|
||||||
|
}
|
||||||
|
}
|
177
src/test/org/apache/sqoop/cloud/tools/CloudTestDataSet.java
Normal file
177
src/test/org/apache/sqoop/cloud/tools/CloudTestDataSet.java
Normal file
@ -0,0 +1,177 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.tools;
|
||||||
|
|
||||||
|
import static java.util.Arrays.asList;
|
||||||
|
import static org.apache.sqoop.testutil.BaseSqoopTestCase.timeFromString;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class CloudTestDataSet {
|
||||||
|
|
||||||
|
private final String[] columnNames = {"ID", "SUPERHERO", "COMICS", "DEBUT"};
|
||||||
|
private final String[] columnTypes = {"INT", "VARCHAR(25)", "VARCHAR(25)", "INT"};
|
||||||
|
private final String[] columnNamesForMerge = {"DEBUT", "SUPERHERO1", "SUPERHERO2", "RECORD_DATE"};
|
||||||
|
private final String[] columnTypesForMerge = {"INT", "VARCHAR(25)", "VARCHAR(25)", "TIMESTAMP"};
|
||||||
|
private final String initialTimestampForMerge = "2018-07-23 15:00:00.000";
|
||||||
|
private final String newTimestampForMerge = "2018-08-16 16:30:09.000";
|
||||||
|
private final String expectedInitialTimestampForMerge = "2018-07-23 15:00:00.0";
|
||||||
|
private final String expectedNewTimestampForMerge = "2018-08-16 16:30:09.0";
|
||||||
|
|
||||||
|
public List<String[]> getInputData() {
|
||||||
|
List<String[]> data = new ArrayList<>();
|
||||||
|
data.add(new String[]{"1", "'Ironman'", "'Marvel'", "1963"});
|
||||||
|
data.add(new String[]{"2", "'Wonder Woman'", "'DC'", "1941"});
|
||||||
|
data.add(new String[]{"3", "'Batman'", "'DC'", "1939"});
|
||||||
|
data.add(new String[]{"4", "'Hulk'", "'Marvel'", "1962"});
|
||||||
|
return data;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExtraInputData() {
|
||||||
|
return new String[]{"5", "'Black Widow'", "'Marvel'", "1964"};
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<List<Object>> getInitialInputDataForMerge() {
|
||||||
|
return Arrays.asList(
|
||||||
|
Arrays.asList(1940, "Black Widow", "Falcon", initialTimestampForMerge),
|
||||||
|
Arrays.asList(1974, "Iron Fist", "The Punisher", initialTimestampForMerge));
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<List<Object>> getNewInputDataForMerge() {
|
||||||
|
return Arrays.asList(
|
||||||
|
Arrays.asList(1962, "Spiderman", "Thor", newTimestampForMerge),
|
||||||
|
Arrays.asList(1974, "Wolverine", "The Punisher", newTimestampForMerge));
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedTextOutput() {
|
||||||
|
return new String[]{
|
||||||
|
"1,Ironman,Marvel,1963",
|
||||||
|
"2,Wonder Woman,DC,1941",
|
||||||
|
"3,Batman,DC,1939",
|
||||||
|
"4,Hulk,Marvel,1962"
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getExpectedTextOutputAsList() {
|
||||||
|
return Arrays.asList(getExpectedTextOutput());
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedExtraTextOutput() {
|
||||||
|
return new String[]{
|
||||||
|
"5,Black Widow,Marvel,1964"
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedTextOutputBeforeMerge() {
|
||||||
|
return new String[]{
|
||||||
|
"1940,Black Widow,Falcon," + expectedInitialTimestampForMerge,
|
||||||
|
"1974,Iron Fist,The Punisher," + expectedInitialTimestampForMerge
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedTextOutputAfterMerge() {
|
||||||
|
return new String[]{
|
||||||
|
"1940,Black Widow,Falcon," + expectedInitialTimestampForMerge,
|
||||||
|
"1962,Spiderman,Thor," + expectedNewTimestampForMerge,
|
||||||
|
"1974,Wolverine,The Punisher," + expectedNewTimestampForMerge
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedSequenceFileOutput() {
|
||||||
|
return new String[]{
|
||||||
|
"1,Ironman,Marvel,1963\n",
|
||||||
|
"2,Wonder Woman,DC,1941\n",
|
||||||
|
"3,Batman,DC,1939\n",
|
||||||
|
"4,Hulk,Marvel,1962\n"
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedExtraSequenceFileOutput() {
|
||||||
|
return new String[]{
|
||||||
|
"5,Black Widow,Marvel,1964\n"
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedAvroOutput() {
|
||||||
|
return new String[]{
|
||||||
|
"{\"ID\": 1, \"SUPERHERO\": \"Ironman\", \"COMICS\": \"Marvel\", \"DEBUT\": 1963}",
|
||||||
|
"{\"ID\": 2, \"SUPERHERO\": \"Wonder Woman\", \"COMICS\": \"DC\", \"DEBUT\": 1941}",
|
||||||
|
"{\"ID\": 3, \"SUPERHERO\": \"Batman\", \"COMICS\": \"DC\", \"DEBUT\": 1939}",
|
||||||
|
"{\"ID\": 4, \"SUPERHERO\": \"Hulk\", \"COMICS\": \"Marvel\", \"DEBUT\": 1962}"
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getExpectedExtraAvroOutput() {
|
||||||
|
return new String[]{
|
||||||
|
"{\"ID\": 5, \"SUPERHERO\": \"Black Widow\", \"COMICS\": \"Marvel\", \"DEBUT\": 1964}"
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getExpectedParquetOutput() {
|
||||||
|
return asList(
|
||||||
|
"1,Ironman,Marvel,1963",
|
||||||
|
"2,Wonder Woman,DC,1941",
|
||||||
|
"3,Batman,DC,1939",
|
||||||
|
"4,Hulk,Marvel,1962");
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getExpectedParquetOutputAfterAppend() {
|
||||||
|
return asList(
|
||||||
|
"1,Ironman,Marvel,1963",
|
||||||
|
"2,Wonder Woman,DC,1941",
|
||||||
|
"3,Batman,DC,1939",
|
||||||
|
"4,Hulk,Marvel,1962",
|
||||||
|
"5,Black Widow,Marvel,1964");
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getExpectedParquetOutputWithTimestampColumn() {
|
||||||
|
return asList(
|
||||||
|
"1940,Black Widow,Falcon," + timeFromString(initialTimestampForMerge),
|
||||||
|
"1974,Iron Fist,The Punisher," + timeFromString(initialTimestampForMerge));
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getExpectedParquetOutputWithTimestampColumnAfterMerge() {
|
||||||
|
return asList(
|
||||||
|
"1940,Black Widow,Falcon," + timeFromString(initialTimestampForMerge),
|
||||||
|
"1962,Spiderman,Thor," + timeFromString(newTimestampForMerge),
|
||||||
|
"1974,Wolverine,The Punisher," + timeFromString(newTimestampForMerge));
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getColumnNames() {
|
||||||
|
return columnNames;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getColumnTypes() {
|
||||||
|
return columnTypes;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getColumnNamesForMerge() {
|
||||||
|
return columnNamesForMerge;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String[] getColumnTypesForMerge() {
|
||||||
|
return columnTypesForMerge;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getInitialTimestampForMerge() {
|
||||||
|
return initialTimestampForMerge;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,50 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.cloud.tools;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
|
||||||
|
import java.io.BufferedReader;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStreamReader;
|
||||||
|
import java.nio.charset.Charset;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
public class CredentialGenerator {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(CredentialGenerator.class.getName());
|
||||||
|
|
||||||
|
public Iterable<String> invokeGeneratorCommand(String generatorCommand) throws IOException {
|
||||||
|
ProcessBuilder processBuilder = new ProcessBuilder("/bin/sh", "-c", generatorCommand);
|
||||||
|
Process process = processBuilder.start();
|
||||||
|
|
||||||
|
try (
|
||||||
|
InputStreamReader inputStreamReader = new InputStreamReader(process.getInputStream(), Charset.forName("UTF-8"));
|
||||||
|
BufferedReader bufferedReader = new BufferedReader(inputStreamReader);
|
||||||
|
) {
|
||||||
|
String output = bufferedReader.readLine();
|
||||||
|
|
||||||
|
return Arrays.asList(output.split(" "));
|
||||||
|
} catch (IOException ioE) {
|
||||||
|
LOG.error("Issue invoking generating credentials", ioE);
|
||||||
|
throw new RuntimeException(ioE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -1,112 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.AvroTestUtils;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3AvroImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3AvroImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpTargetDir() {
|
|
||||||
S3TestUtils.tearDownS3ImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsAvroDataFileWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws IOException {
|
|
||||||
String[] args = getArgsWithAsAvroDataFileOption();
|
|
||||||
runImport(args);
|
|
||||||
AvroTestUtils.verify(S3TestUtils.getExpectedAvroOutput(), s3Client.getConf(), S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsAvroDataFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
|
||||||
String[] args = getArgsWithAsAvroDataFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getArgsWithAsAvroDataFileAndDeleteTargetDirOption();
|
|
||||||
runImport(args);
|
|
||||||
AvroTestUtils.verify(S3TestUtils.getExpectedAvroOutput(), s3Client.getConf(), S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsAvroDataFileWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
|
||||||
String[] args = getArgsWithAsAvroDataFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(args);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsAvroDataFileOption() {
|
|
||||||
return S3TestUtils.getArgsForS3UnitTestsWithFileFormatOption(this, s3CredentialGenerator, "as-avrodatafile");
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsAvroDataFileAndDeleteTargetDirOption() {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-avrodatafile");
|
|
||||||
builder.withOption("delete-target-dir");
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,135 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.hive.minicluster.HiveMiniCluster;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.HiveServer2TestUtil;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.util.BlockJUnit4ClassRunnerWithParametersFactory;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.runner.RunWith;
|
|
||||||
import org.junit.runners.Parameterized;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.tool.BaseSqoopTool.FMT_PARQUETFILE_ARG;
|
|
||||||
import static org.apache.sqoop.tool.BaseSqoopTool.FMT_TEXTFILE_ARG;
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
@RunWith(Parameterized.class)
|
|
||||||
@Parameterized.UseParametersRunnerFactory(BlockJUnit4ClassRunnerWithParametersFactory.class)
|
|
||||||
public class TestS3ExternalHiveTableImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "fileFormatArg = {0}, expectedResult = {1}")
|
|
||||||
public static Iterable<? extends Object> parameters() {
|
|
||||||
return Arrays.asList(new Object[] {FMT_TEXTFILE_ARG, S3TestUtils.getExpectedTextOutputAsList()},
|
|
||||||
new Object[] {FMT_PARQUETFILE_ARG, S3TestUtils.getExpectedParquetOutput()});
|
|
||||||
}
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3ExternalHiveTableImport.class.getName());
|
|
||||||
|
|
||||||
private String fileFormatArg;
|
|
||||||
|
|
||||||
private List<String> expectedResult;
|
|
||||||
|
|
||||||
public TestS3ExternalHiveTableImport(String fileFormatArg, List<String> expectedResult) {
|
|
||||||
this.fileFormatArg = fileFormatArg;
|
|
||||||
this.expectedResult = expectedResult;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
private HiveMiniCluster hiveMiniCluster;
|
|
||||||
|
|
||||||
private static HiveServer2TestUtil hiveServer2TestUtil;
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
hiveMiniCluster = S3TestUtils.setupS3ExternalHiveTableImportTestCase(s3CredentialGenerator);
|
|
||||||
hiveServer2TestUtil = new HiveServer2TestUtil(hiveMiniCluster.getUrl());
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpTargetDir() {
|
|
||||||
S3TestUtils.tearDownS3ExternalHiveTableImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
if (hiveMiniCluster != null) {
|
|
||||||
hiveMiniCluster.stop();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportIntoExternalHiveTable() throws IOException {
|
|
||||||
String[] args = getExternalHiveTableImportArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> rows = hiveServer2TestUtil.loadCsvRowsFromTable(getTableName());
|
|
||||||
assertEquals(rows, expectedResult);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3CreateAndImportIntoExternalHiveTable() throws IOException {
|
|
||||||
String[] args = getExternalHiveTableImportArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> rows = hiveServer2TestUtil.loadCsvRowsFromTable(S3TestUtils.HIVE_EXTERNAL_TABLE_NAME);
|
|
||||||
assertEquals(rows, expectedResult);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getExternalHiveTableImportArgs(boolean createHiveTable) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator, fileFormatArg);
|
|
||||||
builder = S3TestUtils.addExternalHiveTableImportArgs(builder, hiveMiniCluster.getUrl());
|
|
||||||
if(createHiveTable) {
|
|
||||||
builder = S3TestUtils.addCreateHiveTableArgs(builder);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,241 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.s3a.Constants;
|
|
||||||
import org.apache.hadoop.security.alias.CredentialShell;
|
|
||||||
import org.apache.hadoop.util.ToolRunner;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.testutil.TextFileTestUtils;
|
|
||||||
import org.apache.sqoop.util.BlockJUnit4ClassRunnerWithParametersFactory;
|
|
||||||
import org.apache.sqoop.util.password.CredentialProviderHelper;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.AfterClass;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.ClassRule;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.contrib.java.lang.system.EnvironmentVariables;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
import org.junit.runner.RunWith;
|
|
||||||
import org.junit.runners.Parameterized;
|
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.file.Files;
|
|
||||||
import java.util.Arrays;
|
|
||||||
|
|
||||||
import static junit.framework.TestCase.fail;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
@RunWith(Parameterized.class)
|
|
||||||
@Parameterized.UseParametersRunnerFactory(BlockJUnit4ClassRunnerWithParametersFactory.class)
|
|
||||||
public class TestS3ImportWithHadoopCredProvider extends ImportJobTestCase {
|
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "credentialProviderPathProperty = {0}")
|
|
||||||
public static Iterable<? extends Object> parameters() {
|
|
||||||
return Arrays.asList(CredentialProviderHelper.HADOOP_CREDENTIAL_PROVIDER_PATH,
|
|
||||||
CredentialProviderHelper.S3A_CREDENTIAL_PROVIDER_PATH);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3ImportWithHadoopCredProvider.class.getName());
|
|
||||||
|
|
||||||
private String credentialProviderPathProperty;
|
|
||||||
|
|
||||||
public TestS3ImportWithHadoopCredProvider(String credentialProviderPathProperty) {
|
|
||||||
this.credentialProviderPathProperty = credentialProviderPathProperty;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private static String providerPathDefault;
|
|
||||||
private static String providerPathEnv;
|
|
||||||
private static String providerPathPwdFile;
|
|
||||||
|
|
||||||
@ClassRule
|
|
||||||
public static final EnvironmentVariables environmentVariables
|
|
||||||
= new EnvironmentVariables();
|
|
||||||
private static File providerFileDefault;
|
|
||||||
private static File providerFileEnvPwd;
|
|
||||||
private static File providerFilePwdFile;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
private static final String PASSWORD_FILE_NAME = "password-file.txt";
|
|
||||||
private static final String HADOOP_CREDSTORE_PASSWORD_ENV_NAME = "HADOOP_CREDSTORE_PASSWORD";
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws Exception {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
generateTempProviderFileNames();
|
|
||||||
fillCredentialProviderDefault();
|
|
||||||
fillCredentialProviderPwdFile();
|
|
||||||
fillCredentialProviderEnv();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
environmentVariables.clear(HADOOP_CREDSTORE_PASSWORD_ENV_NAME);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpTargetDir() {
|
|
||||||
S3TestUtils.tearDownS3ImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@AfterClass
|
|
||||||
public static void deleteTemporaryCredFiles() {
|
|
||||||
deleteFileOnExit(providerFileDefault);
|
|
||||||
deleteFileOnExit(providerFileEnvPwd);
|
|
||||||
deleteFileOnExit(providerFilePwdFile);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderDefaultSucceeds() throws Exception {
|
|
||||||
runImport(getArgs(providerPathDefault,false, null));
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderEnvSucceeds() throws Exception {
|
|
||||||
setHadoopCredStorePwdEnvVar();
|
|
||||||
runImport(getArgs(providerPathEnv,false, null));
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderPwdFileSucceeds() throws Exception {
|
|
||||||
runImport(getArgs(providerPathPwdFile,true, PASSWORD_FILE_NAME));
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderWithNoProviderPathFails() throws Exception {
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(getArgs(null,false, null));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderWithNoEnvFails() throws Exception {
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(getArgs(providerPathEnv,false, null));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderWithWrongPwdFileFails() throws Exception {
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(getArgs(providerPathPwdFile,true, "wrong-password-file.txt"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCredentialProviderWithNoPwdFileFails() throws Exception {
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(getArgs(providerPathPwdFile,true, null));
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgs(String providerPath, boolean withPwdFile, String pwdFile) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForHadoopCredProviderS3UnitTests(this);
|
|
||||||
|
|
||||||
builder.withProperty(credentialProviderPathProperty, providerPath);
|
|
||||||
if (withPwdFile) {
|
|
||||||
builder.withProperty(CredentialProviderHelper.CREDENTIAL_PROVIDER_PASSWORD_FILE, pwdFile);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void fillCredentialProviderDefault() throws Exception {
|
|
||||||
fillCredentialProvider(new Configuration(), providerPathDefault);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void fillCredentialProviderEnv() throws Exception {
|
|
||||||
setHadoopCredStorePwdEnvVar();
|
|
||||||
fillCredentialProvider(new Configuration(), providerPathEnv);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void fillCredentialProviderPwdFile() throws Exception {
|
|
||||||
Configuration conf = new Configuration();
|
|
||||||
conf.set(CredentialProviderHelper.CREDENTIAL_PROVIDER_PASSWORD_FILE, PASSWORD_FILE_NAME);
|
|
||||||
fillCredentialProvider(conf, providerPathPwdFile);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void generateTempProviderFileNames() throws IOException {
|
|
||||||
providerFileDefault = Files.createTempFile("test-default-pwd-", ".jceks").toFile();
|
|
||||||
boolean deleted = providerFileDefault.delete();
|
|
||||||
providerFileEnvPwd = Files.createTempFile("test-env-pwd-", ".jceks").toFile();
|
|
||||||
deleted &= providerFileEnvPwd.delete();
|
|
||||||
providerFilePwdFile = Files.createTempFile("test-file-pwd-", ".jceks").toFile();
|
|
||||||
deleted &= providerFilePwdFile.delete();
|
|
||||||
if (!deleted) {
|
|
||||||
fail("Could not delete temporary provider files");
|
|
||||||
}
|
|
||||||
providerPathDefault = "jceks://file/" + providerFileDefault.getAbsolutePath();
|
|
||||||
providerPathEnv = "jceks://file/" + providerFileEnvPwd.getAbsolutePath();
|
|
||||||
providerPathPwdFile = "jceks://file/" + providerFilePwdFile.getAbsolutePath();
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void runCredentialProviderCreateCommand(String command, Configuration conf) throws Exception {
|
|
||||||
ToolRunner.run(conf, new CredentialShell(), command.split(" "));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String getCreateCommand(String credentialKey, String credentialValue, String providerPath) {
|
|
||||||
return "create " + credentialKey + " -value " + credentialValue + " -provider " + providerPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void fillCredentialProvider(Configuration conf, String providerPath) throws Exception {
|
|
||||||
runCredentialProviderCreateCommand(getCreateCommand(Constants.ACCESS_KEY, s3CredentialGenerator.getS3AccessKey(), providerPath), conf);
|
|
||||||
runCredentialProviderCreateCommand(getCreateCommand(Constants.SECRET_KEY, s3CredentialGenerator.getS3SecretKey(), providerPath), conf);
|
|
||||||
|
|
||||||
if (s3CredentialGenerator.getS3SessionToken() != null) {
|
|
||||||
runCredentialProviderCreateCommand(getCreateCommand(Constants.SESSION_TOKEN, s3CredentialGenerator.getS3SessionToken(), providerPath), conf);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void setHadoopCredStorePwdEnvVar() {
|
|
||||||
environmentVariables.set(HADOOP_CREDSTORE_PASSWORD_ENV_NAME, "credProviderPwd");
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void deleteFileOnExit(File file) {
|
|
||||||
if (file != null) {
|
|
||||||
file.deleteOnExit();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,133 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.AvroTestUtils;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3IncrementalAppendAvroImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3IncrementalAppendAvroImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpOutputDirectories() {
|
|
||||||
S3TestUtils.tearDownS3IncrementalImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsAvroDataFileWhenNoNewRowIsImported() throws IOException {
|
|
||||||
String[] args = getArgsWithAsAvroDataFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsAvroDataFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.failIfOutputFilePathContainingPatternExists(s3Client, MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsAvroDataFile() throws IOException {
|
|
||||||
String[] args = getArgsWithAsAvroDataFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsAvroDataFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
AvroTestUtils.verify(S3TestUtils.getExpectedExtraAvroOutput(), s3Client.getConf(), S3TestUtils.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsAvroDataFileWithMapreduceOutputBasenameProperty() throws IOException {
|
|
||||||
String[] args = getArgsWithAsAvroDataFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsAvroDataFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
AvroTestUtils.verify(S3TestUtils.getExpectedExtraAvroOutput(), s3Client.getConf(), S3TestUtils.getTargetDirPath(), S3TestUtils.CUSTOM_MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsAvroDataFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-avrodatafile");
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getIncrementalAppendArgsWithAsAvroDataFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-avrodatafile");
|
|
||||||
builder = S3TestUtils.addIncrementalAppendImportArgs(builder);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,140 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.util.ParquetReader;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3IncrementalAppendParquetImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3IncrementalAppendParquetImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpOutputDirectories() {
|
|
||||||
S3TestUtils.tearDownS3IncrementalImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsParquetFileWhenNoNewRowIsImported() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutput(), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsParquetFile() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutputAfterAppend(), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsParquetFileWithMapreduceOutputBasenameProperty() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsParquetFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.failIfOutputFilePathContainingPatternDoesNotExists(s3Client, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutputAfterAppend(), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-parquetfile");
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getIncrementalAppendArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-parquetfile");
|
|
||||||
builder = S3TestUtils.addIncrementalAppendImportArgs(builder);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,133 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.testutil.SequenceFileTestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3IncrementalAppendSequenceFileImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3IncrementalAppendSequenceFileImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpOutputDirectories() {
|
|
||||||
S3TestUtils.tearDownS3IncrementalImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsSequenceFileWhenNoNewRowIsImported() throws Exception {
|
|
||||||
String[] args = getArgsWithAsSequenceFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsSequenceFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.failIfOutputFilePathContainingPatternExists(s3Client, MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsSequenceFile() throws Exception {
|
|
||||||
String[] args = getArgsWithAsSequenceFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsSequenceFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
SequenceFileTestUtils.verify(this, S3TestUtils.getExpectedExtraSequenceFileOutput(), s3Client, S3TestUtils.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsSequenceFileWithMapreduceOutputBasenameProperty() throws Exception {
|
|
||||||
String[] args = getArgsWithAsSequenceFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgsWithAsSequenceFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
SequenceFileTestUtils.verify(this, S3TestUtils.getExpectedExtraSequenceFileOutput(), s3Client, S3TestUtils.getTargetDirPath(), S3TestUtils.CUSTOM_MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsSequenceFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-sequencefile");
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getIncrementalAppendArgsWithAsSequenceFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-sequencefile");
|
|
||||||
builder = S3TestUtils.addIncrementalAppendImportArgs(builder);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,131 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.testutil.TextFileTestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3IncrementalAppendTextImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3IncrementalAppendTextImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpOutputDirectories() {
|
|
||||||
S3TestUtils.tearDownS3IncrementalImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsTextFileWhenNoNewRowIsImported() throws IOException {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.failIfOutputFilePathContainingPatternExists(s3Client, MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsTextFile() throws IOException {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedExtraTextOutput(), s3Client, S3TestUtils.getTargetDirPath(), MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalAppendAsTextFileWithMapreduceOutputBasenameProperty() throws IOException {
|
|
||||||
String[] args = getArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTable(this, S3TestUtils.getExtraInputData());
|
|
||||||
|
|
||||||
args = getIncrementalAppendArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedExtraTextOutput(), s3Client, S3TestUtils.getTargetDirPath(), S3TestUtils.CUSTOM_MAP_OUTPUT_FILE_00001);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgs(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTests(this, s3CredentialGenerator);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getIncrementalAppendArgs(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTests(this, s3CredentialGenerator);
|
|
||||||
builder = S3TestUtils.addIncrementalAppendImportArgs(builder);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,146 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.util.ParquetReader;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3IncrementalMergeParquetImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3IncrementalMergeParquetImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInitialInputDataForMerge(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpOutputDirectories() {
|
|
||||||
S3TestUtils.tearDownS3IncrementalImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalMergeAsParquetFileWhenNoNewRowIsImported() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
clearTable(getTableName());
|
|
||||||
|
|
||||||
args = getIncrementalMergeArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutputWithTimestampColumn(this), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalMergeAsParquetFile() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
clearTable(getTableName());
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTableForMerge(this, S3TestUtils.getNewInputDataForMerge());
|
|
||||||
|
|
||||||
args = getIncrementalMergeArgsWithAsParquetFileOption(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutputWithTimestampColumnAfterMerge(this), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalMergeAsParquetFileWithMapreduceOutputBasenameProperty() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
clearTable(getTableName());
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTableForMerge(this, S3TestUtils.getNewInputDataForMerge());
|
|
||||||
|
|
||||||
args = getIncrementalMergeArgsWithAsParquetFileOption(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
S3TestUtils.failIfOutputFilePathContainingPatternDoesNotExists(s3Client, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutputWithTimestampColumnAfterMerge(this), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-parquetfile");
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getIncrementalMergeArgsWithAsParquetFileOption(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-parquetfile");
|
|
||||||
builder = S3TestUtils.addIncrementalMergeImportArgs(builder);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,137 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.testutil.TextFileTestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3IncrementalMergeTextImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3IncrementalMergeTextImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInitialInputDataForMerge(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpOutputDirectories() {
|
|
||||||
S3TestUtils.tearDownS3IncrementalImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalMergeAsTextFileWhenNoNewRowIsImported() throws Exception {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
clearTable(getTableName());
|
|
||||||
|
|
||||||
args = getIncrementalMergeArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutputBeforeMerge(), s3Client, S3TestUtils.getTargetDirPath(), REDUCE_OUTPUT_FILE_00000);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalMergeAsTextFile() throws Exception {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
clearTable(getTableName());
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTableForMerge(this, S3TestUtils.getNewInputDataForMerge());
|
|
||||||
|
|
||||||
args = getIncrementalMergeArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutputAfterMerge(), s3Client, S3TestUtils.getTargetDirPath(), REDUCE_OUTPUT_FILE_00000);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3IncrementalMergeAsTextFileWithMapreduceOutputBasenameProperty() throws Exception {
|
|
||||||
String[] args = getArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
clearTable(getTableName());
|
|
||||||
|
|
||||||
S3TestUtils.insertInputDataIntoTableForMerge(this, S3TestUtils.getNewInputDataForMerge());
|
|
||||||
|
|
||||||
args = getIncrementalMergeArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutputAfterMerge(), s3Client, S3TestUtils.getTargetDirPath(), S3TestUtils.CUSTOM_REDUCE_OUTPUT_FILE_00000);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgs(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTests(this, s3CredentialGenerator);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getIncrementalMergeArgs(boolean withMapreduceOutputBasenameProperty) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTests(this, s3CredentialGenerator);
|
|
||||||
builder = S3TestUtils.addIncrementalMergeImportArgs(builder);
|
|
||||||
if (withMapreduceOutputBasenameProperty) {
|
|
||||||
builder.withProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY, S3TestUtils.MAPREDUCE_OUTPUT_BASENAME);
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,119 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.util.ParquetReader;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3ParquetImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3ParquetImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpTargetDir() {
|
|
||||||
S3TestUtils.tearDownS3ImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsParquetFileWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutput(), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsParquetFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getArgsWithAsParquetFileAndDeleteTargetDirOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
List<String> result = new ParquetReader(S3TestUtils.getTargetDirPath(), s3Client.getConf()).readAllInCsvSorted();
|
|
||||||
assertEquals(S3TestUtils.getExpectedParquetOutput(), result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsParquetFileWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
|
||||||
String[] args = getArgsWithAsParquetFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(args);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsParquetFileOption() {
|
|
||||||
return S3TestUtils.getArgsForS3UnitTestsWithFileFormatOption(this, s3CredentialGenerator, "as-parquetfile");
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsParquetFileAndDeleteTargetDirOption() {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-parquetfile");
|
|
||||||
builder.withOption("delete-target-dir");
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,112 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.testutil.SequenceFileTestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3SequenceFileImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3SequenceFileImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpTargetDir() {
|
|
||||||
S3TestUtils.tearDownS3ImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsSequenceFileWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws Exception {
|
|
||||||
String[] args = getArgsWithAsSequenceFileOption();
|
|
||||||
runImport(args);
|
|
||||||
SequenceFileTestUtils.verify(this, S3TestUtils.getExpectedSequenceFileOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsSequenceFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
|
||||||
String[] args = getArgsWithAsSequenceFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getArgsWithAsSequenceFileAndDeleteTargetDirOption();
|
|
||||||
runImport(args);
|
|
||||||
SequenceFileTestUtils.verify(this, S3TestUtils.getExpectedSequenceFileOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsSequenceWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws Exception {
|
|
||||||
String[] args = getArgsWithAsSequenceFileOption();
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(args);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsSequenceFileOption() {
|
|
||||||
return S3TestUtils.getArgsForS3UnitTestsWithFileFormatOption(this, s3CredentialGenerator, "as-sequencefile");
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithAsSequenceFileAndDeleteTargetDirOption() {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(this,
|
|
||||||
s3CredentialGenerator,"as-sequencefile");
|
|
||||||
builder.withOption("delete-target-dir");
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,144 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.s3;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.sqoop.testcategories.thirdpartytest.S3Test;
|
|
||||||
import org.apache.sqoop.testutil.ArgumentArrayBuilder;
|
|
||||||
import org.apache.sqoop.testutil.DefaultS3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.ImportJobTestCase;
|
|
||||||
import org.apache.sqoop.testutil.S3CredentialGenerator;
|
|
||||||
import org.apache.sqoop.testutil.S3TestUtils;
|
|
||||||
import org.apache.sqoop.testutil.TextFileTestUtils;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
@Category(S3Test.class)
|
|
||||||
public class TestS3TextImport extends ImportJobTestCase {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
TestS3TextImport.class.getName());
|
|
||||||
|
|
||||||
private static S3CredentialGenerator s3CredentialGenerator;
|
|
||||||
|
|
||||||
private FileSystem s3Client;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setupS3Credentials() throws IOException {
|
|
||||||
String generatorCommand = S3TestUtils.getGeneratorCommand();
|
|
||||||
if (generatorCommand != null) {
|
|
||||||
s3CredentialGenerator = new DefaultS3CredentialGenerator(generatorCommand);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() throws IOException {
|
|
||||||
S3TestUtils.runTestCaseOnlyIfS3CredentialsAreSet(s3CredentialGenerator);
|
|
||||||
super.setUp();
|
|
||||||
S3TestUtils.createTestTableFromInputData(this);
|
|
||||||
s3Client = S3TestUtils.setupS3ImportTestCase(s3CredentialGenerator);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void cleanUpTargetDir() {
|
|
||||||
S3TestUtils.tearDownS3ImportTestCase(s3Client);
|
|
||||||
super.tearDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testImportWithoutDeleteTargetDirOptionWhenTargetDirDoesNotExist() throws IOException {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testImportWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getArgsWithDeleteTargetOption(false);
|
|
||||||
runImport(args);
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testImportWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
|
||||||
String[] args = getArgs(false);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(args);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsTextFile() throws IOException {
|
|
||||||
String[] args = getArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsTextFileWithDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
|
||||||
String[] args = getArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
args = getArgsWithDeleteTargetOption(true);
|
|
||||||
runImport(args);
|
|
||||||
TextFileTestUtils.verify(S3TestUtils.getExpectedTextOutput(), s3Client, S3TestUtils.getTargetDirPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testS3ImportAsTextFileWithoutDeleteTargetDirOptionWhenTargetDirAlreadyExists() throws IOException {
|
|
||||||
String[] args = getArgs(true);
|
|
||||||
runImport(args);
|
|
||||||
|
|
||||||
thrown.expect(IOException.class);
|
|
||||||
runImport(args);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgs(boolean withAsTextFileOption) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTests(this, s3CredentialGenerator);
|
|
||||||
if (withAsTextFileOption) {
|
|
||||||
builder.withOption("as-textfile");
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
private String[] getArgsWithDeleteTargetOption(boolean withAsTextFileOption) {
|
|
||||||
ArgumentArrayBuilder builder = S3TestUtils.getArgumentArrayBuilderForS3UnitTests(this, s3CredentialGenerator);
|
|
||||||
builder.withOption("delete-target-dir");
|
|
||||||
if (withAsTextFileOption) {
|
|
||||||
builder.withOption("as-textfile");
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,91 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.testutil;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import java.io.BufferedReader;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.InputStreamReader;
|
|
||||||
import java.nio.charset.Charset;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public class DefaultS3CredentialGenerator extends S3CredentialGenerator {
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
DefaultS3CredentialGenerator.class.getName());
|
|
||||||
|
|
||||||
/**
|
|
||||||
* By calling this constructor the {@link #generateS3Credentials(String)} method is called and the values of the
|
|
||||||
* returned map are assigned to the corresponding fields of the instantiated class.
|
|
||||||
*
|
|
||||||
* @param generatorCommand String containing the command to generate S3 credentials
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public DefaultS3CredentialGenerator(String generatorCommand) throws IOException {
|
|
||||||
super(generatorCommand);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/**
|
|
||||||
* Executes the given command under /bin/sh and reads space separated S3 credentials from
|
|
||||||
* the first line of standard output in the following order: access key, secret key and session token
|
|
||||||
* (the latter one only in case of temporary credentials).
|
|
||||||
*
|
|
||||||
* @param {@inheritDoc}
|
|
||||||
* @return {@inheritDoc}
|
|
||||||
* @throws {@inheritDoc}
|
|
||||||
*/
|
|
||||||
protected Map<String, String> generateS3Credentials(String generatorCommand) throws IOException {
|
|
||||||
ProcessBuilder processBuilder = new ProcessBuilder("/bin/sh", "-c", generatorCommand);
|
|
||||||
Process process = processBuilder.start();
|
|
||||||
String output;
|
|
||||||
Map<String, String> credentials = new HashMap<String, String>();
|
|
||||||
|
|
||||||
try (
|
|
||||||
InputStreamReader inputStreamReader = new InputStreamReader(process.getInputStream(), Charset.forName("UTF-8"));
|
|
||||||
BufferedReader bufferedReader = new BufferedReader(inputStreamReader);
|
|
||||||
) {
|
|
||||||
output = bufferedReader.readLine();
|
|
||||||
|
|
||||||
if (output != null) {
|
|
||||||
|
|
||||||
String[] splitOutput = output.split(" ");
|
|
||||||
|
|
||||||
credentials.put(S3_ACCESS_KEY, splitOutput[0]);
|
|
||||||
credentials.put(S3_SECRET_KEY, splitOutput[1]);
|
|
||||||
|
|
||||||
if (splitOutput.length > 2) {
|
|
||||||
credentials.put(S3_SESSION_TOKEN, splitOutput[2]);
|
|
||||||
}
|
|
||||||
|
|
||||||
} else {
|
|
||||||
LOG.info("No S3 credential generator command is given or output of the command is null thus S3 tests are being skipped.");
|
|
||||||
}
|
|
||||||
|
|
||||||
} catch (IOException ioE) {
|
|
||||||
LOG.error("Issue with generating S3 credentials", ioE);
|
|
||||||
throw new RuntimeException(ioE);
|
|
||||||
}
|
|
||||||
|
|
||||||
return credentials;
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,71 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.testutil;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public abstract class S3CredentialGenerator {
|
|
||||||
|
|
||||||
protected static final String S3_ACCESS_KEY = "S3_ACCESS_KEY";
|
|
||||||
protected static final String S3_SECRET_KEY = "S3_SECRET_KEY";
|
|
||||||
protected static final String S3_SESSION_TOKEN = "S3_SESSION_TOKEN";
|
|
||||||
|
|
||||||
protected String s3AccessKey;
|
|
||||||
protected String s3SecretKey;
|
|
||||||
protected String s3SessionToken;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* By calling this constructor the {@link #generateS3Credentials(String)} method is called and the values of the
|
|
||||||
* returned map are assigned to the corresponding fields of the instantiated class.
|
|
||||||
*
|
|
||||||
* @param generatorCommand String containing the command to generate S3 credentials
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public S3CredentialGenerator(String generatorCommand) throws IOException {
|
|
||||||
Map<String, String> s3Credentials = generateS3Credentials(generatorCommand);
|
|
||||||
if (s3Credentials != null) {
|
|
||||||
s3AccessKey = s3Credentials.get(S3_ACCESS_KEY);
|
|
||||||
s3SecretKey = s3Credentials.get(S3_SECRET_KEY);
|
|
||||||
s3SessionToken = s3Credentials.get(S3_SESSION_TOKEN);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Executes the given S3 credential generator command and builds a map containing the credentials
|
|
||||||
*
|
|
||||||
* @param generatorCommand String containing the command to execute
|
|
||||||
* @return Map containing S3 credentials by keys {@link #S3_ACCESS_KEY}, {@link #S3_SECRET_KEY} and {@link #S3_SESSION_TOKEN}
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
protected abstract Map<String, String> generateS3Credentials(String generatorCommand) throws IOException;
|
|
||||||
|
|
||||||
public String getS3AccessKey() {
|
|
||||||
return s3AccessKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getS3SecretKey() {
|
|
||||||
return s3SecretKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getS3SessionToken() {
|
|
||||||
return s3SessionToken;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,439 +0,0 @@
|
|||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.sqoop.testutil;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.fs.s3a.Constants;
|
|
||||||
import org.apache.sqoop.hive.minicluster.HiveMiniCluster;
|
|
||||||
import org.apache.sqoop.hive.minicluster.NoAuthenticationConfiguration;
|
|
||||||
import org.apache.sqoop.util.FileSystemUtil;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.UUID;
|
|
||||||
|
|
||||||
import static java.util.Arrays.asList;
|
|
||||||
import static org.apache.sqoop.util.AppendUtils.MAPREDUCE_OUTPUT_BASENAME_PROPERTY;
|
|
||||||
import static org.junit.Assert.fail;
|
|
||||||
import static org.junit.Assume.assumeNotNull;
|
|
||||||
|
|
||||||
public class S3TestUtils {
|
|
||||||
|
|
||||||
private static final String PROPERTY_GENERATOR_COMMAND = "s3.generator.command";
|
|
||||||
private static final String PROPERTY_BUCKET_URL = "s3.bucket.url";
|
|
||||||
|
|
||||||
private static final String TEMPORARY_CREDENTIALS_PROVIDER_CLASS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
|
|
||||||
|
|
||||||
private static final String BUCKET_TEMP_DIR = "/tmp/";
|
|
||||||
|
|
||||||
private static final String TARGET_DIR_NAME_PREFIX = "/testdir";
|
|
||||||
private static final String HIVE_EXTERNAL_DIR_NAME_PREFIX = "/externaldir";
|
|
||||||
|
|
||||||
private static final String TEMPORARY_ROOTDIR_SUFFIX = "_temprootdir";
|
|
||||||
|
|
||||||
public static final String HIVE_EXTERNAL_TABLE_NAME = "test_external_table";
|
|
||||||
|
|
||||||
private static String targetDirName;
|
|
||||||
private static String hiveExternalTableDirName;
|
|
||||||
|
|
||||||
private static final String[] COLUMN_NAMES = {"ID", "SUPERHERO", "COMICS", "DEBUT"};
|
|
||||||
private static final String[] COLUMN_TYPES = { "INT", "VARCHAR(25)", "VARCHAR(25)", "INT"};
|
|
||||||
|
|
||||||
private static final String[] COLUMN_NAMES_FOR_MERGE = { "DEBUT", "SUPERHERO1", "SUPERHERO2", "RECORD_DATE"};
|
|
||||||
private static final String[] COLUMN_TYPES_FOR_MERGE = { "INT", "VARCHAR(25)", "VARCHAR(25)", "TIMESTAMP"};
|
|
||||||
private static final String INITIAL_TIMESTAMP_FOR_MERGE = "2018-07-23 15:00:00.000";
|
|
||||||
private static final String NEW_TIMESTAMP_FOR_MERGE = "2018-08-16 16:30:09.000";
|
|
||||||
private static final String EXPECTED_INITIAL_TIMESTAMP_FOR_MERGE = "2018-07-23 15:00:00.0";
|
|
||||||
private static final String EXPECTED_NEW_TIMESTAMP_FOR_MERGE = "2018-08-16 16:30:09.0";
|
|
||||||
|
|
||||||
public static final String MAPREDUCE_OUTPUT_BASENAME = "custom";
|
|
||||||
public static final String CUSTOM_MAP_OUTPUT_FILE_00001 = MAPREDUCE_OUTPUT_BASENAME + "-m-00001";
|
|
||||||
public static final String CUSTOM_REDUCE_OUTPUT_FILE_00000 = MAPREDUCE_OUTPUT_BASENAME + "-r-00000";
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
S3TestUtils.class.getName());
|
|
||||||
|
|
||||||
public static String getGeneratorCommand() {
|
|
||||||
return System.getProperty(PROPERTY_GENERATOR_COMMAND);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String getPropertyBucketUrl() {
|
|
||||||
return System.getProperty(PROPERTY_BUCKET_URL);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String getTemporaryCredentialsProviderClass() {
|
|
||||||
return TEMPORARY_CREDENTIALS_PROVIDER_CLASS;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String generateUniqueDirName(String dirPrefix) {
|
|
||||||
String uuid = UUID.randomUUID().toString();
|
|
||||||
return dirPrefix + "-" + uuid;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void resetTargetDirName() {
|
|
||||||
targetDirName = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void resetHiveExternalDirName() {
|
|
||||||
hiveExternalTableDirName = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String getTargetDirName() {
|
|
||||||
return targetDirName;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String getHiveExternalTableDirName() {
|
|
||||||
return hiveExternalTableDirName;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Path getTargetDirPath() {
|
|
||||||
String targetPathString = getBucketTempDirPath() + getTargetDirName();
|
|
||||||
return new Path(targetPathString);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static Path getBucketTempDirPath() {
|
|
||||||
String targetPathString = getPropertyBucketUrl() + BUCKET_TEMP_DIR;
|
|
||||||
return new Path(targetPathString);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Path getExternalTableDirPath() {
|
|
||||||
String externalTableDir = getBucketTempDirPath() + getHiveExternalTableDirName();
|
|
||||||
return new Path(externalTableDir);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void runTestCaseOnlyIfS3CredentialsAreSet(S3CredentialGenerator s3CredentialGenerator) {
|
|
||||||
assumeNotNull(s3CredentialGenerator);
|
|
||||||
assumeNotNull(s3CredentialGenerator.getS3AccessKey());
|
|
||||||
assumeNotNull(s3CredentialGenerator.getS3SecretKey());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static FileSystem setupS3ImportTestCase(S3CredentialGenerator s3CredentialGenerator) throws IOException {
|
|
||||||
Configuration hadoopConf = new Configuration();
|
|
||||||
setS3CredentialsInConf(hadoopConf, s3CredentialGenerator);
|
|
||||||
setHadoopConfigParametersForS3UnitTests(hadoopConf);
|
|
||||||
|
|
||||||
FileSystem s3Client = FileSystem.get(hadoopConf);
|
|
||||||
|
|
||||||
targetDirName = generateUniqueDirName(TARGET_DIR_NAME_PREFIX);
|
|
||||||
|
|
||||||
cleanUpDirectory(s3Client, getTargetDirPath());
|
|
||||||
|
|
||||||
return s3Client;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void setS3CredentialsInConf(Configuration conf,
|
|
||||||
S3CredentialGenerator s3CredentialGenerator) {
|
|
||||||
conf.set(Constants.ACCESS_KEY, s3CredentialGenerator.getS3AccessKey());
|
|
||||||
conf.set(Constants.SECRET_KEY, s3CredentialGenerator.getS3SecretKey());
|
|
||||||
|
|
||||||
if (s3CredentialGenerator.getS3SessionToken() != null) {
|
|
||||||
conf.set(Constants.SESSION_TOKEN, s3CredentialGenerator.getS3SessionToken());
|
|
||||||
conf.set(Constants.AWS_CREDENTIALS_PROVIDER, TEMPORARY_CREDENTIALS_PROVIDER_CLASS);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void setHadoopConfigParametersForS3UnitTests(Configuration hadoopConf) {
|
|
||||||
// Default filesystem needs to be set to S3 for the output verification phase
|
|
||||||
hadoopConf.set("fs.defaultFS", getPropertyBucketUrl());
|
|
||||||
|
|
||||||
// FileSystem has a static cache that should be disabled during tests to make sure
|
|
||||||
// Sqoop relies on the S3 credentials set via the -D system properties.
|
|
||||||
// For details please see SQOOP-3383
|
|
||||||
hadoopConf.setBoolean("fs.s3a.impl.disable.cache", true);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static HiveMiniCluster setupS3ExternalHiveTableImportTestCase(S3CredentialGenerator s3CredentialGenerator) {
|
|
||||||
hiveExternalTableDirName = generateUniqueDirName(HIVE_EXTERNAL_DIR_NAME_PREFIX);
|
|
||||||
HiveMiniCluster hiveMiniCluster = new HiveMiniCluster(new NoAuthenticationConfiguration());
|
|
||||||
hiveMiniCluster.start();
|
|
||||||
S3TestUtils.setS3CredentialsInConf(hiveMiniCluster.getConfig(), s3CredentialGenerator);
|
|
||||||
|
|
||||||
return hiveMiniCluster;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder getArgumentArrayBuilderForS3UnitTests(BaseSqoopTestCase testCase,
|
|
||||||
S3CredentialGenerator s3CredentialGenerator) {
|
|
||||||
ArgumentArrayBuilder builder = new ArgumentArrayBuilder();
|
|
||||||
return builder.withCommonHadoopFlags()
|
|
||||||
.withProperty(Constants.ACCESS_KEY, s3CredentialGenerator.getS3AccessKey())
|
|
||||||
.withProperty(Constants.SECRET_KEY, s3CredentialGenerator.getS3SecretKey())
|
|
||||||
.withProperty(Constants.SESSION_TOKEN, s3CredentialGenerator.getS3SessionToken())
|
|
||||||
.withProperty(Constants.AWS_CREDENTIALS_PROVIDER, getTemporaryCredentialsProviderClass())
|
|
||||||
.withOption("connect", testCase.getConnectString())
|
|
||||||
.withOption("num-mappers", "1")
|
|
||||||
.withOption("table", testCase.getTableName())
|
|
||||||
.withOption("target-dir", getTargetDirPath().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder getArgumentArrayBuilderForHadoopCredProviderS3UnitTests(BaseSqoopTestCase testCase) {
|
|
||||||
|
|
||||||
ArgumentArrayBuilder builder = new ArgumentArrayBuilder();
|
|
||||||
return builder.withCommonHadoopFlags()
|
|
||||||
.withProperty("fs.s3a.impl.disable.cache", "true")
|
|
||||||
.withProperty(Constants.AWS_CREDENTIALS_PROVIDER, getTemporaryCredentialsProviderClass())
|
|
||||||
.withOption("connect", testCase.getConnectString())
|
|
||||||
.withOption("num-mappers", "1")
|
|
||||||
.withOption("table", testCase.getTableName())
|
|
||||||
.withOption("target-dir", getTargetDirPath().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder getArgumentArrayBuilderForS3UnitTestsWithFileFormatOption(BaseSqoopTestCase testCase,
|
|
||||||
S3CredentialGenerator s3CredentialGenerator,
|
|
||||||
String fileFormat) {
|
|
||||||
ArgumentArrayBuilder builder = getArgumentArrayBuilderForS3UnitTests(testCase, s3CredentialGenerator);
|
|
||||||
builder.withOption(fileFormat);
|
|
||||||
return builder;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getArgsForS3UnitTestsWithFileFormatOption(BaseSqoopTestCase testCase,
|
|
||||||
S3CredentialGenerator s3CredentialGenerator,
|
|
||||||
String fileFormat) {
|
|
||||||
ArgumentArrayBuilder builder = getArgumentArrayBuilderForS3UnitTests(testCase, s3CredentialGenerator);
|
|
||||||
builder.withOption(fileFormat);
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder addExternalHiveTableImportArgs(ArgumentArrayBuilder builder,
|
|
||||||
String hs2Url) {
|
|
||||||
return builder
|
|
||||||
.withOption("hive-import")
|
|
||||||
.withOption("hs2-url", hs2Url)
|
|
||||||
.withOption("external-table-dir", getExternalTableDirPath().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder addCreateHiveTableArgs(ArgumentArrayBuilder builder) {
|
|
||||||
return builder
|
|
||||||
.withOption("create-hive-table")
|
|
||||||
.withOption("hive-table", HIVE_EXTERNAL_TABLE_NAME);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static Path getTemporaryRootDirPath() {
|
|
||||||
return new Path(getTargetDirPath().toString() + TEMPORARY_ROOTDIR_SUFFIX);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder addIncrementalAppendImportArgs(ArgumentArrayBuilder builder) {
|
|
||||||
return builder
|
|
||||||
.withOption("incremental", "append")
|
|
||||||
.withOption("check-column", "ID")
|
|
||||||
.withOption("last-value", "4")
|
|
||||||
.withOption("temporary-rootdir", getTemporaryRootDirPath().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ArgumentArrayBuilder addIncrementalMergeImportArgs(ArgumentArrayBuilder builder) {
|
|
||||||
return builder
|
|
||||||
.withOption("incremental", "lastmodified")
|
|
||||||
.withOption("check-column", "RECORD_DATE")
|
|
||||||
.withOption("merge-key", "DEBUT")
|
|
||||||
.withOption("last-value", INITIAL_TIMESTAMP_FOR_MERGE)
|
|
||||||
.withOption("temporary-rootdir", getTemporaryRootDirPath().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<String[]> getInputData() {
|
|
||||||
List<String[]> data = new ArrayList<>();
|
|
||||||
data.add(new String[]{"1", "'Ironman'", "'Marvel'", "1963"});
|
|
||||||
data.add(new String[]{"2", "'Wonder Woman'", "'DC'", "1941"});
|
|
||||||
data.add(new String[]{"3", "'Batman'", "'DC'", "1939"});
|
|
||||||
data.add(new String[]{"4", "'Hulk'", "'Marvel'", "1962"});
|
|
||||||
return data;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExtraInputData() {
|
|
||||||
return new String[]{"5", "'Black Widow'", "'Marvel'", "1964"};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<List<Object>> getInitialInputDataForMerge() {
|
|
||||||
return Arrays.<List<Object>>asList(
|
|
||||||
Arrays.<Object>asList(1940, "Black Widow", "Falcon", INITIAL_TIMESTAMP_FOR_MERGE),
|
|
||||||
Arrays.<Object>asList(1974, "Iron Fist", "The Punisher", INITIAL_TIMESTAMP_FOR_MERGE));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<List<Object>> getNewInputDataForMerge() {
|
|
||||||
return Arrays.<List<Object>>asList(
|
|
||||||
Arrays.<Object>asList(1962, "Spiderman", "Thor", NEW_TIMESTAMP_FOR_MERGE),
|
|
||||||
Arrays.<Object>asList(1974, "Wolverine", "The Punisher", NEW_TIMESTAMP_FOR_MERGE));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void createTestTableFromInputData(BaseSqoopTestCase testCase) {
|
|
||||||
List<String[]> inputData = getInputData();
|
|
||||||
testCase.createTableWithColTypesAndNames(COLUMN_NAMES, COLUMN_TYPES, new String[0]);
|
|
||||||
for (String[] dataRow : inputData) {
|
|
||||||
insertInputDataIntoTable(testCase, dataRow);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void insertInputDataIntoTable(BaseSqoopTestCase testCase, String[] inputData) {
|
|
||||||
testCase.insertIntoTable(COLUMN_NAMES, COLUMN_TYPES, inputData);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void createTestTableFromInitialInputDataForMerge(BaseSqoopTestCase testCase) {
|
|
||||||
testCase.createTableWithRecordsWithColTypesAndNames(COLUMN_NAMES_FOR_MERGE, COLUMN_TYPES_FOR_MERGE, getInitialInputDataForMerge());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void insertInputDataIntoTableForMerge(BaseSqoopTestCase testCase, List<List<Object>> inputData) {
|
|
||||||
testCase.insertRecordsIntoTableWithColTypesAndNames(COLUMN_NAMES_FOR_MERGE, COLUMN_TYPES_FOR_MERGE, inputData);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedTextOutput() {
|
|
||||||
return new String[] {
|
|
||||||
"1,Ironman,Marvel,1963",
|
|
||||||
"2,Wonder Woman,DC,1941",
|
|
||||||
"3,Batman,DC,1939",
|
|
||||||
"4,Hulk,Marvel,1962"
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<String> getExpectedTextOutputAsList() {
|
|
||||||
return Arrays.asList(getExpectedTextOutput());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedExtraTextOutput() {
|
|
||||||
return new String[] {
|
|
||||||
"5,Black Widow,Marvel,1964"
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedTextOutputBeforeMerge() {
|
|
||||||
return new String[] {
|
|
||||||
"1940,Black Widow,Falcon," + EXPECTED_INITIAL_TIMESTAMP_FOR_MERGE,
|
|
||||||
"1974,Iron Fist,The Punisher," + EXPECTED_INITIAL_TIMESTAMP_FOR_MERGE
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedTextOutputAfterMerge() {
|
|
||||||
return new String[] {
|
|
||||||
"1940,Black Widow,Falcon," + EXPECTED_INITIAL_TIMESTAMP_FOR_MERGE,
|
|
||||||
"1962,Spiderman,Thor," + EXPECTED_NEW_TIMESTAMP_FOR_MERGE,
|
|
||||||
"1974,Wolverine,The Punisher," + EXPECTED_NEW_TIMESTAMP_FOR_MERGE
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedSequenceFileOutput() {
|
|
||||||
return new String[] {
|
|
||||||
"1,Ironman,Marvel,1963\n",
|
|
||||||
"2,Wonder Woman,DC,1941\n",
|
|
||||||
"3,Batman,DC,1939\n",
|
|
||||||
"4,Hulk,Marvel,1962\n"
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedExtraSequenceFileOutput() {
|
|
||||||
return new String[] {
|
|
||||||
"5,Black Widow,Marvel,1964\n"
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedAvroOutput() {
|
|
||||||
return new String[] {
|
|
||||||
"{\"ID\": 1, \"SUPERHERO\": \"Ironman\", \"COMICS\": \"Marvel\", \"DEBUT\": 1963}",
|
|
||||||
"{\"ID\": 2, \"SUPERHERO\": \"Wonder Woman\", \"COMICS\": \"DC\", \"DEBUT\": 1941}",
|
|
||||||
"{\"ID\": 3, \"SUPERHERO\": \"Batman\", \"COMICS\": \"DC\", \"DEBUT\": 1939}",
|
|
||||||
"{\"ID\": 4, \"SUPERHERO\": \"Hulk\", \"COMICS\": \"Marvel\", \"DEBUT\": 1962}"
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static String[] getExpectedExtraAvroOutput() {
|
|
||||||
return new String[] {
|
|
||||||
"{\"ID\": 5, \"SUPERHERO\": \"Black Widow\", \"COMICS\": \"Marvel\", \"DEBUT\": 1964}"
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<String> getExpectedParquetOutput() {
|
|
||||||
return asList(
|
|
||||||
"1,Ironman,Marvel,1963",
|
|
||||||
"2,Wonder Woman,DC,1941",
|
|
||||||
"3,Batman,DC,1939",
|
|
||||||
"4,Hulk,Marvel,1962");
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<String> getExpectedParquetOutputAfterAppend() {
|
|
||||||
return asList(
|
|
||||||
"1,Ironman,Marvel,1963",
|
|
||||||
"2,Wonder Woman,DC,1941",
|
|
||||||
"3,Batman,DC,1939",
|
|
||||||
"4,Hulk,Marvel,1962",
|
|
||||||
"5,Black Widow,Marvel,1964");
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<String> getExpectedParquetOutputWithTimestampColumn(BaseSqoopTestCase testCase) {
|
|
||||||
return asList(
|
|
||||||
"1940,Black Widow,Falcon," + testCase.timeFromString(INITIAL_TIMESTAMP_FOR_MERGE),
|
|
||||||
"1974,Iron Fist,The Punisher," + testCase.timeFromString(INITIAL_TIMESTAMP_FOR_MERGE));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static List<String> getExpectedParquetOutputWithTimestampColumnAfterMerge(BaseSqoopTestCase testCase) {
|
|
||||||
return asList(
|
|
||||||
"1940,Black Widow,Falcon," + testCase.timeFromString(INITIAL_TIMESTAMP_FOR_MERGE),
|
|
||||||
"1962,Spiderman,Thor," + testCase.timeFromString(NEW_TIMESTAMP_FOR_MERGE),
|
|
||||||
"1974,Wolverine,The Punisher," + testCase.timeFromString(NEW_TIMESTAMP_FOR_MERGE));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void failIfOutputFilePathContainingPatternExists(FileSystem s3Client, String pattern) throws IOException {
|
|
||||||
List<Path> outputFilesWithPathContainingPattern = FileSystemUtil.findFilesWithPathContainingPattern(getTargetDirPath(),
|
|
||||||
s3Client.getConf(), pattern);
|
|
||||||
if (outputFilesWithPathContainingPattern.size() != 0) {
|
|
||||||
fail("No output file was expected with pattern" + pattern);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void failIfOutputFilePathContainingPatternDoesNotExists(FileSystem s3Client, String pattern) throws IOException {
|
|
||||||
List<Path> outputFilesWithPathContainingPattern = FileSystemUtil.findFilesWithPathContainingPattern(getTargetDirPath(),
|
|
||||||
s3Client.getConf(), pattern);
|
|
||||||
if (outputFilesWithPathContainingPattern.size() == 0) {
|
|
||||||
fail("No output file was found with pattern" + pattern);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void cleanUpDirectory(FileSystem s3Client, Path directoryPath) {
|
|
||||||
|
|
||||||
try {
|
|
||||||
if (s3Client.exists(directoryPath)) {
|
|
||||||
s3Client.delete(directoryPath, true);
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
LOG.error("Issue with cleaning up directory", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void cleanUpTargetDir(FileSystem s3Client) {
|
|
||||||
cleanUpDirectory(s3Client, getTargetDirPath());
|
|
||||||
resetTargetDirName();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void tearDownS3ImportTestCase(FileSystem s3Client) {
|
|
||||||
cleanUpTargetDir(s3Client);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void tearDownS3IncrementalImportTestCase(FileSystem s3Client) {
|
|
||||||
cleanUpDirectory(s3Client, getTemporaryRootDirPath());
|
|
||||||
cleanUpTargetDir(s3Client);
|
|
||||||
System.clearProperty(MAPREDUCE_OUTPUT_BASENAME_PROPERTY);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void tearDownS3ExternalHiveTableImportTestCase(FileSystem s3Client) {
|
|
||||||
cleanUpTargetDir(s3Client);
|
|
||||||
cleanUpDirectory(s3Client, getExternalTableDirPath());
|
|
||||||
resetHiveExternalDirName();
|
|
||||||
}
|
|
||||||
}
|
|
Loading…
Reference in New Issue
Block a user