[ASTERIXDB-3519][EXT]: Support dynamic prefixes on HDFS
- user model changes: add dynamic prefixes for HDFS datasets
- storage format changes: no
- interface changes: no
details:
- Support dynamic prefixes on HDFS
- Support include, exclude and definition parameters in HDFS
- Parameters 'path' or 'definition' can be used to specify prefixes
Change-Id: I1bdbcd44c059f64f2da436a40ac3f59293442cf2
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19066
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Hussain Towaileb <hussainht@gmail.com>
Tested-by: Hussain Towaileb <hussainht@gmail.com>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 17cf78a..1d21ce2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -5755,11 +5755,28 @@
}
}
+ /**
+ * Normalizes the value of the adapter and ensures that it is supported
+ *
+ * @param details external details
+ * @param sourceLoc source location
+ */
+ private void normalizeAdapters(ExternalDetailsDecl details, SourceLocation sourceLoc) throws CompilationException {
+ String adapter = details.getAdapter();
+ Optional<String> normalizedAdapter = ExternalDataConstants.EXTERNAL_READ_ADAPTERS.stream()
+ .filter(k -> k.equalsIgnoreCase(adapter)).findFirst();
+ if (normalizedAdapter.isEmpty()) {
+ throw CompilationException.create(ErrorCode.UNKNOWN_ADAPTER, sourceLoc, adapter);
+ }
+ details.setAdapter(normalizedAdapter.get());
+ }
+
protected void validateExternalDatasetProperties(ExternalDetailsDecl externalDetails,
Map<String, String> properties, SourceLocation srcLoc, MetadataTransactionContext mdTxnCtx,
IApplicationContext appCtx, MetadataProvider metadataProvider)
throws AlgebricksException, HyracksDataException {
// Validate adapter specific properties
+ normalizeAdapters(externalDetails, srcLoc);
String adapter = externalDetails.getAdapter();
Map<String, String> details = new HashMap<>(properties);
details.put(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE, adapter);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java
index 2658804..24191d8 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java
@@ -25,6 +25,7 @@
import java.util.regex.Pattern;
import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.TestCase;
import org.apache.commons.io.IOUtils;
import com.fasterxml.jackson.databind.JsonNode;
@@ -59,6 +60,22 @@
return res;
}
+ @Override
+ public ExtractedResult executeSqlppUpdateOrDdl(String statement, TestCaseContext.OutputFormat outputFormat,
+ TestCase.CompilationUnit cUnit) throws Exception {
+ Matcher dvMatcher = usePattern.matcher(statement);
+ String dv = "";
+ if (dvMatcher.find()) {
+ dv = dvMatcher.group(2) + ".";
+ }
+ Matcher dsMatcher = loadPattern.matcher(statement);
+ Matcher upsertMatcher = upsertPattern.matcher(statement);
+ ExtractedResult res = super.executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(SQLPP), cUnit);
+ analyzeFromRegex(dsMatcher, dv, 3);
+ analyzeFromRegex(upsertMatcher, dv, 2);
+ return res;
+ }
+
private void analyzeFromRegex(Matcher m, String dv, int pos) throws Exception {
while (m.find()) {
String ds = m.group(pos);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
index 0a96943..bef2f97 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
@@ -34,6 +34,10 @@
public static final String S3_TEMPLATE_DEFAULT = "(\"accessKeyId\"=\"" + S3_ACCESS_KEY_ID_DEFAULT + "\"),\n"
+ "(\"secretAccessKey\"=\"" + S3_SECRET_ACCESS_KEY_DEFAULT + "\"),\n" + "(\"region\"=\"" + S3_REGION_DEFAULT
+ "\"),\n" + "(\"serviceEndpoint\"=\"" + S3_SERVICE_ENDPOINT_DEFAULT + "\")";
+ public static final String S3_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS =
+ "\"accessKeyId\":\"" + S3_ACCESS_KEY_ID_DEFAULT + "\",\n" + "\"secretAccessKey\":\""
+ + S3_SECRET_ACCESS_KEY_DEFAULT + "\",\n" + "\"region\":\"" + S3_REGION_DEFAULT + "\",\n"
+ + "\"serviceEndpoint\":\"" + S3_SERVICE_ENDPOINT_DEFAULT + "\"";
// Azure blob storage constants and placeholders
public static class Azure {
@@ -89,5 +93,10 @@
public static final String KERBEROS_PASSWORD_DEFAULT = "hdfspassword";
public static final String KERBEROS_REALM_DEFAULT = "EXAMPLE.COM";
public static final String KERBEROS_KDC_DEFAULT = "localhost:8800";
+ public static final String HDFS_ENDPOINT_DEFAULT = "hdfs://localhost:31888";
+
+ public static final String HDFS_TEMPLATE_DEFAULT = "(\"hdfs\"=\"" + HDFS_ENDPOINT_DEFAULT + "\")";
+ public static final String HDFS_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS =
+ "\"hdfs\":\"" + HDFS_ENDPOINT_DEFAULT + "\"";
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 3de0cd7..8b71d89 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -1274,7 +1274,11 @@
if (isDmlRecoveryTest && statement.contains("nc1://")) {
statement = statement.replaceAll("nc1://", "127.0.0.1://../../../../../../asterix-app/");
}
- executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
+ if (cUnit.getPlaceholder().isEmpty()) {
+ executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
+ } else {
+ executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit), cUnit);
+ }
break;
case "pollget":
case "pollquery":
@@ -2453,7 +2457,7 @@
}
protected boolean noTemplateRequired(String str) {
- return !str.contains("%template%");
+ return !str.contains("%template%") && !str.contains("%template_colons%");
}
protected String applyS3Substitution(String str, List<Placeholder> placeholders) {
@@ -2504,7 +2508,11 @@
}
protected String setS3TemplateDefault(String str) {
- return str.replace("%template%", TestConstants.S3_TEMPLATE_DEFAULT);
+ if (str.contains("%template%")) {
+ return str.replace("%template%", TestConstants.S3_TEMPLATE_DEFAULT);
+ } else {
+ return str.replace("%template_colons%", TestConstants.S3_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS);
+ }
}
protected String applyAzureSubstitution(String str, List<Placeholder> placeholders) {
@@ -2554,7 +2562,12 @@
}
protected String setHDFSTemplateDefault(String str) {
- return str;
+ if (str.contains("%template%")) {
+ return str.replace("%template%", TestConstants.HDFS.HDFS_TEMPLATE_DEFAULT);
+ } else {
+ return str.replace("%template_colons%",
+ TestConstants.HDFS.HDFS_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS);
+ }
}
protected void fail(boolean runDiagnostics, TestCaseContext testCaseCtx, CompilationUnit cUnit,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
index fe3006e..412fbc1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
@@ -18,17 +18,36 @@
*/
package org.apache.asterix.test.runtime;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createAvroFiles;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createAvroFilesRecursively;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFiles;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFilesRecursively;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createDeltaTable;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setDataPaths;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setUploaders;
+import static org.apache.asterix.test.external_dataset.parquet.BinaryFileConverterUtil.DEFAULT_PARQUET_SRC_PATH;
+import static org.apache.hyracks.util.file.FileUtil.joinPath;
import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath;
import static org.apache.iceberg.types.Types.NestedField.required;
+import java.io.FileInputStream;
import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
import java.util.Collection;
import java.util.List;
+import java.util.zip.GZIPOutputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.test.common.TestConstants;
import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils;
import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
@@ -60,34 +79,42 @@
*/
@RunWith(Parameterized.class)
public class SqlppHdfsExecutionTest {
+ private static final String PATH_BASE = joinPath("data");
+ private static final String EXTERNAL_FILTER_DATA_PATH = joinPath(PATH_BASE, "json", "external-filter");
+
protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
- private static DataFile writeFile(String filename, List<Record> records, String location, Schema schema,
- Configuration conf) throws IOException {
+ static Runnable PREPARE_BUCKET;
+
+ private static final String JSON_DATA_PATH = joinPath("data", "json");
+ private static final String CSV_DATA_PATH = joinPath("data", "csv");
+ private static final String TSV_DATA_PATH = joinPath("data", "tsv");
+
+ private static final Configuration CONF = new Configuration();
+
+ private static DataFile writeFile(String filename, List<Record> records, String location, Schema schema)
+ throws IOException {
Path path = new Path(location, filename);
FileFormat fileFormat = FileFormat.fromFileName(filename);
Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename);
FileAppender<Record> fileAppender =
- new GenericAppenderFactory(schema).newAppender(fromPath(path, conf), fileFormat);
+ new GenericAppenderFactory(schema).newAppender(fromPath(path, CONF), fileFormat);
try (FileAppender<Record> appender = fileAppender) {
appender.addAll(records);
}
- return DataFiles.builder(PartitionSpec.unpartitioned()).withInputFile(HadoopInputFile.fromPath(path, conf))
+ return DataFiles.builder(PartitionSpec.unpartitioned()).withInputFile(HadoopInputFile.fromPath(path, CONF))
.withMetrics(fileAppender.metrics()).build();
}
private static void setUpIcebergData() {
- Configuration conf = new Configuration();
- conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, "hdfs://127.0.0.1:31888/");
-
- Tables tables = new HadoopTables(conf);
+ Tables tables = new HadoopTables(CONF);
Schema schema =
new Schema(required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get()));
- String path = "hdfs://localhost:31888/my_table/";
+ String path = TestConstants.HDFS.HDFS_ENDPOINT_DEFAULT + "/my_table/";
Table table = tables.create(schema, PartitionSpec.unpartitioned(),
ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name()), path);
@@ -100,8 +127,7 @@
// load test data
try {
- DataFile file =
- writeFile(FileFormat.PARQUET.addExtension("file"), fileFirstSnapshotRecords, path, schema, conf);
+ DataFile file = writeFile(FileFormat.PARQUET.addExtension("file"), fileFirstSnapshotRecords, path, schema);
table.newAppend().appendFile(file).commit();
} catch (IOException e) {
throw new RuntimeException(e);
@@ -110,8 +136,52 @@
@BeforeClass
public static void setUp() throws Exception {
+ CONF.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, TestConstants.HDFS.HDFS_ENDPOINT_DEFAULT);
LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor(), true);
setUpIcebergData();
+ createBinaryFiles(DEFAULT_PARQUET_SRC_PATH);
+ createBinaryFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
+ createAvroFiles(DEFAULT_PARQUET_SRC_PATH);
+ createAvroFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
+ createDeltaTable();
+ setUpData();
+ }
+
+ private static void setUpData() {
+ setDataPaths(JSON_DATA_PATH, CSV_DATA_PATH, TSV_DATA_PATH);
+ setUploaders(SqlppHdfsExecutionTest::loadPlaygroundData, null, null, null, null);
+
+ PREPARE_BUCKET.run();
+ }
+
+ private static void loadPlaygroundData(String key, String content, boolean fromFile, boolean gzipped) {
+ loadData("/playground/", key, content, fromFile, gzipped);
+ }
+
+ private static void loadData(String prefix, String key, String content, boolean fromFile, boolean gzipped) {
+ try {
+ try (FileSystem fs = FileSystem.get(CONF)) {
+ Path path = new Path(prefix + key);
+ if (!fromFile) {
+ try (FSDataOutputStream out = fs.create(path)) {
+ out.writeBytes(content);
+ }
+ } else {
+ if (!gzipped) {
+ try (FSDataOutputStream out = fs.create(path); InputStream in = new FileInputStream(content)) {
+ IOUtils.copy(in, out);
+ }
+ } else {
+ try (FSDataOutputStream out = fs.create(path);
+ GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) {
+ gzipOutputStream.write(Files.readAllBytes(Paths.get(content)));
+ }
+ }
+ }
+ }
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
}
@AfterClass
@@ -121,6 +191,7 @@
@Parameters(name = "SqlppHdfsExecutionTest {index}: {0}")
public static Collection<Object[]> tests() throws Exception {
+ PREPARE_BUCKET = ExternalDatasetTestUtils::preparePlaygroundContainer;
return LangExecutionUtil.tests("only_sqlpp_hdfs.xml", "testsuite_sqlpp_hdfs.xml");
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp
index c04f2fe..606457a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp
@@ -21,15 +21,12 @@
COPY (
SELECT id, name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "delimiter")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "delimiter")
AS (id bigint, name STRING, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp
index b19cb4b..2208b2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp
@@ -19,17 +19,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="false"),
-("delimiter"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/delimiter"),
+ %template%,
+ ("header"="false"),
+ ("delimiter"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/delimiter"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp
index 05d7b83..12a7978 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp
@@ -23,17 +23,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopyWrong(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopyWrong(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="false"),
-("delimiter"=","),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/delimiter"),
+ %template%,
+ ("header"="false"),
+ ("delimiter"=","),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/delimiter"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp
index cc0c717..df3ee42 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp
@@ -21,15 +21,12 @@
COPY (
SELECT id, name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "header")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "header")
AS (id bigint, name STRING, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|",
"header":"true"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp
index f65d0d4..2516a7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp
@@ -19,17 +19,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("delimiter"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/header"),
+ %template%,
+ ("header"="true"),
+ ("delimiter"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/header"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp
index 3b13ef0..157a9ff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp
@@ -26,15 +26,12 @@
COPY (
SELECT id, null name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "default", "null")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "default", "null")
AS (id bigint, name STRING, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|",
"header":"true"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp
index 1d6cd22..0754048 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp
@@ -19,17 +19,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopyDefaultNull(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopyDefaultNull(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("delimiter"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/default/null"),
+ %template%,
+ ("header"="true"),
+ ("delimiter"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/default/null"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp
index 46ee936..f02b7b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
SELECT id, null name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "custom", "null")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "custom", "null")
AS (id bigint, name STRING, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|",
"header":"true",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp
index 82f048d..29b53e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp
@@ -23,17 +23,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopyCustomNull(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopyCustomNull(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("delimiter"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/custom/null"),
+ %template%,
+ ("header"="true"),
+ ("delimiter"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/custom/null"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp
index 1fcf3cd..7d7d447 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
SELECT id, null name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "notUnknown")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "notUnknown")
AS (id bigint, name STRING not unknown, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|",
"header":"true",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp
index eb00e4d..c0f62b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp
@@ -23,17 +23,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopyNotUnknown(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopyNotUnknown(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("delimiter"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/notUnknown"),
+ %template%,
+ ("header"="true"),
+ ("delimiter"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/notUnknown"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp
index 8dd62e7..dcfeef0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp
@@ -26,15 +26,12 @@
COPY (
SELECT id, name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "escape")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "escape")
AS (id bigint, name STRING, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"header":"true",
"escape":"|"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp
index 2ac9258..2ddd4c5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp
@@ -19,17 +19,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("escape"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/escape"),
+ %template%,
+ ("header"="true"),
+ ("escape"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/escape"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp
index b468831..94ce946 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp
@@ -21,15 +21,12 @@
COPY (
SELECT id, name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "escape", "1")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "escape", "1")
AS (id bigint, name STRING, amount float, accountNumber double)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"header":"true",
"escape":"|",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp
index 9224e3c..aaf9211 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp
@@ -19,17 +19,13 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy1(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopy1(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("escape"="|"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/escape/1"),
+ %template%,
+ ("header"="true"),
+ ("escape"="|"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/escape/1"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp
index cddab9e..b9a6e9a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp
@@ -26,15 +26,12 @@
COPY (
SELECT id, null name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "simple-csv", "1")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "simple-csv", "1")
TYPE ( {id: bigint, name: string?, amount: float, accountNumber: double} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|",
"header":"true",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp
index c38e775d..2c583ba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp
@@ -19,18 +19,14 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy1(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopy1(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("delimiter"="|"),
-("quote"="'"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/simple-csv/1"),
+ %template%,
+ ("header"="true"),
+ ("delimiter"="|"),
+ ("quote"="'"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/simple-csv/1"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp
index 5a8bd3a..7e477be 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp
@@ -27,15 +27,12 @@
COPY (
SELECT id, name, amount, accountNumber FROM TestCollection
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "simple-csv", "2")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "simple-csv", "2")
TYPE ( {id: bigint, name: string?, amount: float, accountNumber: double} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter":"|",
"header":"true",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp
index 07577d0..5eca10c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp
@@ -19,18 +19,14 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy2(ColumnType) USING S3
+CREATE EXTERNAL DATASET DatasetCopy2(ColumnType) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="true"),
-("delimiter"="|"),
-("quote"="'"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/simple-csv/2"),
+ %template%,
+ ("header"="true"),
+ ("delimiter"="|"),
+ ("quote"="'"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/simple-csv/2"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp
index ac5b0b9..d138157 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp
@@ -21,14 +21,11 @@
COPY (
SELECT "123" as id
) toWriter
-TO S3
-PATH ("copy-to-result", "csv", "type-mismatch")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "type-mismatch")
AS (id bigint)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp
index 8daf039..69d133d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp
@@ -19,16 +19,12 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy(id String) USING S3
+CREATE EXTERNAL DATASET DatasetCopy(id String) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("header"="false"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/csv/type-mismatch"),
+ %template%,
+ ("header"="false"),
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/csv/type-mismatch"),
("format" = "csv"),
("requireVersionChangeDetection"="false"),
("include"="*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp
index 7207aa7..31db6b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp
@@ -20,13 +20,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp
index 55916f3..5ee47cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp
@@ -18,14 +18,11 @@
*/
COPY Customer c
-TO S3
-PATH ("copy-to-result", "default-namespace1")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "default-namespace1")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp
index edb038f..7fae07a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp
@@ -18,18 +18,15 @@
*/
COPY Customer AS c
-TO S3
-PATH ("copy-to-result/default-namespace2", company, "customer", customer_id)
+TO %adapter%
+PATH (%pathprefix% "copy-to-result/default-namespace2", company, "customer", customer_id)
OVER (
PARTITION BY c.company company,
c.customer_id customer_id
)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp
index ee75bd7..8464574 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp
@@ -21,18 +21,15 @@
SELECT DISTINCT UPPERCASE(c.company) company, c.year
FROM Customer c
) AS toWriter
-TO S3
-PATH ("copy-to-result/default-namespace3", company, year)
+TO %adapter%
+PATH (%pathprefix% "copy-to-result/default-namespace3", company, year)
OVER (
PARTITION BY toWriter.company company,
toWriter.year year
)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp
index 4c7ba29..5e7de72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp
@@ -17,35 +17,26 @@
* under the License.
*/
-CREATE EXTERNAL DATASET CustomerCopy1(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/default-namespace1"),
+CREATE EXTERNAL DATASET CustomerCopy1(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/default-namespace1"),
("embed-filter-values" = "false"),
("format"="json")
);
-CREATE EXTERNAL DATASET CustomerCopy2(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/default-namespace2/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET CustomerCopy2(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/default-namespace2/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
-CREATE EXTERNAL DATASET CustomerCopy3(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/default-namespace3/{company:string}/{year:int}"),
+CREATE EXTERNAL DATASET CustomerCopy3(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/default-namespace3/{company:string}/{year:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp
index 02f6c96..b7dce3a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result-with-over")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result-with-over")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json",
"max-objects-per-file": 2
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp
index c0add71..33e3457 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result-with-over")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result-with-over")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json",
"max-objects-per-file": "hello"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp
index ef6631b..db7509f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp
@@ -22,13 +22,10 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks2")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks2")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp
index 87ebe40..cc02e20 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp
@@ -22,14 +22,11 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
AS (id wrongDataType)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp
index 117fbde..78f752d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
AS (id bigint)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"quote": "ABCD"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp
index aed1ed4..41e1c83 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
AS (id bigint)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"delimiter": "wrongDelimiter"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp
index e51644b..4d20c23 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
AS (id bigint)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"escape": "wrongEscape"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp
index 2509d18..a53e2ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
AS (id bigint)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"record-delimiter": "ABCD"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp
index a995d7a..5d68650 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
TYPE ( { id : int, name : { first : [ string ] } } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"record-delimiter": ","
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp
index e2d7b4f..ceab3aa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
TYPE ( { id : int, name : [ string ] } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"record-delimiter": ","
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp
index 8706be3..6a38cea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp
@@ -22,16 +22,13 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "csv-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
TYPE ( { id : int, name : string } )
AS ( id string )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"csv",
"record-delimiter": ","
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp
index 434801b..5c035cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
+TO %adapter%
PATH (MISSING)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp
index d02486b..af3f9f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
+TO %adapter%
PATH (1 + "hello world")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp
index cc0c4b4..2df776c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp
@@ -20,15 +20,12 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result-with-over")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result-with-over")
OVER()
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp
index a0e04db..be98bcd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result", "duplicate-write")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "duplicate-write")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp
index a0e04db..be98bcd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result", "duplicate-write")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "duplicate-write")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp
index 8eb14c0..926dfaa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp
@@ -25,16 +25,13 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks3")
TYPE ( {id : string, name : string, nested: { first : string, second : string} }
)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp
index 5e8d730..fae01f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp
@@ -24,15 +24,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks4")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks4")
TYPE ( {id : bigint,name : string, nested: { first : string } } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp
index cda2519..21c91dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp
@@ -23,15 +23,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks5")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks5")
TYPE ( { id : bigint, name : string, nested : string } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp
index a6db11d..2508c5c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp
@@ -23,15 +23,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks6")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks6")
TYPE ( {id : bigint,name : { first : string },nested:{first : string,second : string} } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp
index 851559a..bb8a348 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp
@@ -23,15 +23,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks7")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks7")
TYPE ( {id:int} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"row-group-size":"random"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp
index 2f356fb..ac7050c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp
@@ -23,15 +23,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks8")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks8")
TYPE ( {id:int} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"page-size":"random"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp
index f2293e3..a2ae2e4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp
@@ -23,15 +23,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks9")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks9")
TYPE ( { name:string } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"compression":"rar"
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp
index 4f52164..0069616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks10")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks10")
TYPE ( { name: } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"schema":"message schema{"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp
index ca2cfc1..0f9fc7d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks11")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks11")
TYPE ( { id : int , name : binary } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp
index 86e55e5..2531328 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks12")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks12")
TYPE ( { id : int, name : string } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"version" : 3
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp
index f5d7eda..85bec05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks13")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks13")
TYPE ( { id : int, name : string, list : [string]} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"max-schemas" : "yvghc"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp
index 7628044..4614ead 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks14")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks14")
TYPE ( {id : int, name : string, l : int } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"max-schemas" : "15"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp
index 89bfb2a..be198a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp
@@ -31,14 +31,11 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks15")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks15")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp
index ad4fc73..ed2d0d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp
@@ -35,14 +35,11 @@
COPY (
select id,name from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-error-checks16")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks16")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"max-schemas" : "2"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp
index a9fe8cf..2c75a42 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"avro"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp
index 38934b9..6969404 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json",
"compression": "rar"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp
index 66bb709..b528721 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp
@@ -20,17 +20,14 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result", "order-by1")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "order-by1")
-- We do not have a way to determine the output is sorted
-- However, this test ensures that there's no regressions when having ORDER BY only in the OVER-clause
OVER (ORDER BY c.company)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp
index 0380332..182fe2d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp
@@ -19,13 +19,10 @@
USE test;
-CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/order-by1"),
+CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/order-by1"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp
index 1fae58f..fa7ecec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp
@@ -23,17 +23,14 @@
SELECT c.company make
FROM Customer c
) x
-TO S3
-PATH ("copy-to-result", "order-by2")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "order-by2")
-- We do not have a way to determine the output is sorted
-- However, this test ensures that there's no regressions when having ORDER BY only in the OVER-clause
OVER (ORDER BY x.make)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp
index fab6964..7c38e71 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp
@@ -19,13 +19,10 @@
USE test;
-CREATE EXTERNAL DATASET CustomerCarMake1(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/order-by2"),
+CREATE EXTERNAL DATASET CustomerCarMake1(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/order-by2"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp
index dc0a631..b3616f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp
@@ -23,17 +23,14 @@
SELECT c.company make
FROM Customer c
) x
-TO S3
-PATH ("copy-to-result", "order-by3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "order-by3")
-- We do not have a way to determine the output is sorted
-- However, this test ensures that there's no regressions when having ORDER BY only in the OVER-clause
OVER (ORDER BY x.make)
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp
index e3369be..9e800c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp
@@ -19,13 +19,10 @@
USE test;
-CREATE EXTERNAL DATASET CustomerCarMake2(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/order-by2"),
+CREATE EXTERNAL DATASET CustomerCarMake2(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/order-by2"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
index 0012e22..a85e188 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-cover-data-types")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-cover-data-types")
TYPE ( { name : string, id : int, dateType : date, timeType : time, boolType : boolean, doubleType : double, datetimeType : datetime } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
index 310198e..54c9373 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
@@ -25,12 +25,11 @@
-CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING S3
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING %adapter%
(
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("region"="us-west-2"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-cover-data-types/"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-cover-data-types/"),
("include"="*.parquet"),
("requireVersionChangeDetection"="false"),
("format" = "parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp
index 9a652d2..0f33fcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-empty-array")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-empty-array")
TYPE ( { id : int, name : { first : [ string ] } } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
index f17c4d4..93aa8b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
@@ -25,12 +25,11 @@
-CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING S3
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING %adapter%
(
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("region"="us-west-2"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-empty-array/"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-empty-array/"),
("include"="*.parquet"),
("requireVersionChangeDetection"="false"),
("format" = "parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp
index a7e6e16..ea9c327 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp
@@ -22,13 +22,10 @@
COPY (
select c.* from TestCollection c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-heterogeneous")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-heterogeneous")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp
index 95f75c6..0494045 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp
@@ -25,12 +25,11 @@
-CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING S3
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING %adapter%
(
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("region"="us-west-2"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-heterogeneous/"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-heterogeneous/"),
("include"="*.parquet"),
("requireVersionChangeDetection"="false"),
("format" = "parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp
index 948ba17..fd2053e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
SELECT t.* FROM ParitionParquetDataset t
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-partition-heterogeneous" , partitioner_key , "random" )
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-partition-heterogeneous" , partitioner_key , "random" )
OVER ( PARTITION BY toWriter.partitioner_key AS partitioner_key )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"max-schemas" : "10"
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp
index c626270..6e3e17b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp
@@ -22,12 +22,11 @@
-CREATE EXTERNAL DATASET ParitionParquetDatasetCopy(ColumnType2) USING S3
+CREATE EXTERNAL DATASET ParitionParquetDatasetCopy(ColumnType2) USING %adapter%
(
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("region"="us-west-2"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-partition-heterogeneous/"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-partition-heterogeneous/"),
("include"="*.parquet"),
("requireVersionChangeDetection"="false"),
("format" = "parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp
index 4bec537..9b21be7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp
@@ -23,15 +23,12 @@
COPY (
select "123" as id
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-simple")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-simple")
TYPE ( {id:string} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet",
"version" : "2"
};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp
index a5d7789..f1a4798 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp
@@ -20,15 +20,11 @@
USE test;
-CREATE EXTERNAL DATASET DatasetCopy(ColumnType2) USING S3
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType2) USING %adapter%
(
-("accessKeyId"="dummyAccessKey"),
-("secretAccessKey"="dummySecretKey"),
-("sessionToken"="dummySessionToken"),
-("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-simple"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-simple"),
("format" = "parquet"),
("requireVersionChangeDetection"="false"),
("include"="*.parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp
index aed4e09..28872dd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp
@@ -22,8 +22,8 @@
COPY (
SELECT c.* FROM DummyTweetDataset c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-tweet")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-tweet")
TYPE ( {
coordinates: {
coordinates: [
@@ -206,11 +206,8 @@
}
} )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp
index 1cf0c78..5a93999 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp
@@ -22,12 +22,11 @@
-CREATE EXTERNAL DATASET DummyTweetDatasetCopy(ColumnType2) USING S3
+CREATE EXTERNAL DATASET DummyTweetDatasetCopy(ColumnType2) USING %adapter%
(
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("region"="us-west-2"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-tweet/"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-tweet/"),
("include"="*.parquet"),
("requireVersionChangeDetection"="false"),
("format" = "parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp
index 9a1c9a4..ad94373 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp
@@ -22,15 +22,12 @@
COPY (
SELECT c.* FROM NameCommentDataset c
) toWriter
-TO S3
-PATH ("copy-to-result", "parquet-utf8")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-utf8")
TYPE ( { comment:string, id:bigint, name:string } )
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"parquet"
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp
index 4fd41f6..bf71240 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp
@@ -22,12 +22,11 @@
-CREATE EXTERNAL DATASET NameCommentDatasetCopy(ColumnType2) USING S3
+CREATE EXTERNAL DATASET NameCommentDatasetCopy(ColumnType2) USING %adapter%
(
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("region"="us-west-2"),
- ("container"="playground"),
- ("definition"="copy-to-result/parquet-utf8/"),
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/parquet-utf8/"),
("include"="*.parquet"),
("requireVersionChangeDetection"="false"),
("format" = "parquet")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp
@@ -24,13 +24,10 @@
CREATE TYPE OpenType AS {
};
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp
index 94aa7a0..0ee6d29 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp
@@ -20,14 +20,11 @@
USE test;
COPY Customer c
-TO S3
-PATH ("copy-to-result", "simple-write")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "simple-write")
WITH {
- "accessKeyId":"dummyAccessKey",
- "secretAccessKey":"dummySecretKey",
- "region":"us-west-2",
- "serviceEndpoint":"http://127.0.0.1:8001",
- "container":"playground",
+ %template_colons%,
+ %additionalProperties%
"format":"json"
}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp
index 5f03abd..1c34766 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp
@@ -19,13 +19,10 @@
USE test;
-CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING S3 (
- ("accessKeyId"="dummyAccessKey"),
- ("secretAccessKey"="dummySecretKey"),
- ("region"="us-west-2"),
- ("serviceEndpoint"="http://127.0.0.1:8001"),
- ("container"="playground"),
- ("definition"="copy-to-result/simple-write"),
+CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING %adapter% (
+ %template%,
+ %additional_Properties%,
+ ("definition"="%path_prefix%copy-to-result/simple-write"),
("embed-filter-values" = "false"),
("format"="json")
);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
index ee9dc3b..7aaa8b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
@@ -26,6 +26,11 @@
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="simple-write">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">simple-write</output-dir>
</compilation-unit>
</test-case>
@@ -36,41 +41,81 @@
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="default-namespace">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">default-namespace</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-simple">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-simple</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-tweet">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-tweet</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-partition-heterogeneous">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-partition-heterogeneous</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-utf8">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-utf8</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-heterogeneous">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-heterogeneous</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-cover-data-types">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-cover-data-types</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="parquet-empty-array">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-empty-array</output-dir>
</compilation-unit>
</test-case>
@@ -81,11 +126,20 @@
</test-case>
<test-case FilePath="copy-to">
<compilation-unit name="order-by">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">order-by</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to/negative">
<compilation-unit name="early-missing">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">early-missing</output-dir>
<expected-error>ASX0064: Path expression produced a value of type 'missing'. Path must be of type string</expected-error>
<expected-error>ASX0064: Path expression produced a value of type 'null'. Path must be of type string</expected-error>
@@ -101,6 +155,11 @@
<test-case FilePath="copy-to/negative">
<compilation-unit name="non-empty-folder">
<output-dir compare="Text">non-empty-folder</output-dir>
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<expected-error>ASX0062: Cannot write to a non-empty directory 'copy-to-result/duplicate-write'</expected-error>
</compilation-unit>
</test-case>
@@ -113,6 +172,11 @@
</test-case>
<test-case FilePath="copy-to/negative">
<compilation-unit name="supported-adapter-format-compression">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">supported-adapter-format-compression</output-dir>
<expected-error>ASX1188: Unsupported writing adapter 'AZUREBLOB'. Supported adapters: [gcs, hdfs, localfs, s3]</expected-error>
<expected-error>ASX1189: Unsupported writing format 'avro'. Supported formats: [csv, json, parquet]</expected-error>
@@ -121,6 +185,11 @@
</test-case>
<test-case FilePath="copy-to/negative">
<compilation-unit name="parquet-error-checks">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">parquet-error-checks</output-dir>
<expected-error>ASX0037: Type mismatch: expected value of type BINARY, but got the value of type bigint</expected-error>
<expected-error>HYR0132: Extra field in the result, field 'second' does not exist at 'nested' in the schema</expected-error>
@@ -136,16 +205,26 @@
<expected-error>ASX1209: Maximum value allowed for 'max-schemas' is 10. Found 15</expected-error>
<expected-error>HYR0133: Schema could not be inferred, empty types found in the result</expected-error>
<expected-error>HYR0134: Schema Limit exceeded, maximum number of heterogeneous schemas allowed : '2'</expected-error>
- </compilation-unit>
+ </compilation-unit>
</test-case>
<test-case FilePath="copy-to/negative">
<compilation-unit name="empty-over">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">empty-over</output-dir>
<expected-error>ASX1001: Syntax error: OVER-clause cannot be empty</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to/negative">
<compilation-unit name="bad-max-objects-per-file">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">bad-max-objects-per-file</output-dir>
<expected-error>Minimum value allowed for 'max-objects-per-file' is 1000. Found 2</expected-error>
<expected-error>Expected integer value, got hello</expected-error>
@@ -153,6 +232,11 @@
</test-case>
<test-case FilePath="copy-to/negative">
<compilation-unit name="csv-error-checks">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">csv-error-checks</output-dir>
<expected-error>ASX1079: Compilation error: TYPE/AS Expression is required for csv format</expected-error>
<expected-error>ASX1082: Cannot find datatype with name wrongDataType (in line 27, at column 4)</expected-error>
@@ -169,26 +253,51 @@
<test-group name="copy-to/csv">
<test-case FilePath="copy-to/csv">
<compilation-unit name="simple-csv">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">simple-csv</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to/csv">
<compilation-unit name="type-mismatch">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">type-mismatch</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to/csv">
<compilation-unit name="delimiter">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">delimiter</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to/csv">
<compilation-unit name="header">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">header</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="copy-to/csv">
<compilation-unit name="quote-escape">
+ <placeholder name="adapter" value="S3" />
+ <placeholder name="pathprefix" value="" />
+ <placeholder name="path_prefix" value="" />
+ <placeholder name="additionalProperties" value='"container":"playground",' />
+ <placeholder name="additional_Properties" value='("container"="playground")' />
<output-dir compare="Text">quote-escape</output-dir>
</compilation-unit>
</test-case>
@@ -223,16 +332,16 @@
<output-dir compare="Text">common/csv/csv</output-dir>
</compilation-unit>
</test-case><test-case FilePath="external-dataset">
- <compilation-unit name="common/csv/gz">
- <placeholder name="adapter" value="S3" />
- <output-dir compare="Text">common/csv/gz</output-dir>
- </compilation-unit>
- </test-case><test-case FilePath="external-dataset">
- <compilation-unit name="common/csv/mixed">
- <placeholder name="adapter" value="S3" />
- <output-dir compare="Text">common/csv/mixed</output-dir>
- </compilation-unit>
- </test-case>
+ <compilation-unit name="common/csv/gz">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/csv/gz</output-dir>
+ </compilation-unit>
+ </test-case><test-case FilePath="external-dataset">
+ <compilation-unit name="common/csv/mixed">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/csv/mixed</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="external-dataset">
<compilation-unit name="common/tsv/tsv">
<placeholder name="adapter" value="S3" />
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
index d14746a..a5af248 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
@@ -92,5 +92,261 @@
<output-dir compare="Text">parquet-empty-array</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="simple-write">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">simple-write</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="default-namespace">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">default-namespace</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-simple">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-simple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-tweet">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-tweet</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-partition-heterogeneous">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-partition-heterogeneous</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-utf8">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-utf8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-heterogeneous">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-heterogeneous</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-cover-data-types">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-cover-data-types</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="parquet-empty-array">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+ <output-dir compare="Text">parquet-empty-array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to">
+ <compilation-unit name="order-by">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">order-by</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="early-missing">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">early-missing</output-dir>
+ <expected-error>ASX0064: Path expression produced a value of type 'missing'. Path must be of type string</expected-error>
+ <expected-error>ASX0064: Path expression produced a value of type 'null'. Path must be of type string</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="non-empty-folder">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">non-empty-folder</output-dir>
+ <expected-error>ASX0062: Cannot write to a non-empty directory '/playground/copy-to-result/duplicate-write'</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="supported-adapter-format-compression">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">supported-adapter-format-compression</output-dir>
+ <expected-error>ASX1188: Unsupported writing adapter 'AZUREBLOB'. Supported adapters: [gcs, hdfs, localfs, s3]</expected-error>
+ <expected-error>ASX1189: Unsupported writing format 'avro'. Supported formats: [csv, json, parquet]</expected-error>
+ <expected-error>ASX1202: Unsupported compression scheme rar. Supported schemes for json are [gzip]</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="parquet-error-checks">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <output-dir compare="Text">parquet-error-checks</output-dir>
+ <expected-error>ASX0037: Type mismatch: expected value of type BINARY, but got the value of type bigint</expected-error>
+ <expected-error>HYR0132: Extra field in the result, field 'second' does not exist at 'nested' in the schema</expected-error>
+ <expected-error>HYR0131: Result does not follow the schema, group type expected but found primitive type at 'nested'</expected-error>
+ <expected-error>HYR0131: Result does not follow the schema, primitive type expected but found group type at 'name'</expected-error>
+ <expected-error>ASX1206: Storage units expected for the field 'row-group-size' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided 'random'</expected-error>
+ <expected-error>ASX1206: Storage units expected for the field 'page-size' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided 'random'</expected-error>
+ <expected-error>ASX1202: Unsupported compression scheme rar. Supported schemes for parquet are [gzip, snappy, zstd]</expected-error>
+ <expected-error>ASX1001: Syntax error</expected-error>
+ <expected-error>ASX1204: 'binary' type not supported in parquet format</expected-error>
+ <expected-error>ASX1205: Invalid Parquet Writer Version provided '3'. Supported values: [1, 2]</expected-error>
+ <expected-error>ASX0039: Expected integer value, got yvghc (in line 22, at column 6)</expected-error>
+ <expected-error>ASX1209: Maximum value allowed for 'max-schemas' is 10. Found 15</expected-error>
+ <expected-error>HYR0133: Schema could not be inferred, empty types found in the result</expected-error>
+ <expected-error>HYR0134: Schema Limit exceeded, maximum number of heterogeneous schemas allowed : '2'</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="empty-over">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">empty-over</output-dir>
+ <expected-error>ASX1001: Syntax error: OVER-clause cannot be empty</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="bad-max-objects-per-file">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">bad-max-objects-per-file</output-dir>
+ <expected-error>Minimum value allowed for 'max-objects-per-file' is 1000. Found 2</expected-error>
+ <expected-error>Expected integer value, got hello</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/negative">
+ <compilation-unit name="csv-error-checks">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">csv-error-checks</output-dir>
+ <expected-error>ASX1079: Compilation error: TYPE/AS Expression is required for csv format</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name wrongDataType (in line 27, at column 4)</expected-error>
+ <expected-error>ASX3124: 'ABCD' is not a valid quote. The length of a quote should be 1</expected-error>
+ <expected-error>ASX3049: 'wrongDelimiter' is not a valid delimiter. The length of a delimiter should be 1</expected-error>
+ <expected-error>ASX3126: 'wrongEscape' is not a valid escape. The length of a escape should be 1</expected-error>
+ <expected-error>ASX3125: 'ABCD' is not a valid force-quote input. The length of a force-quote input should be 1 character</expected-error>
+ <expected-error>ASX1207: 'object' type not supported in csv format</expected-error>
+ <expected-error>ASX1207: 'array' type not supported in csv format</expected-error>
+ <expected-error>Syntax error: Both 'TYPE()' and 'AS()' are provided. Please use either 'TYPE()' or 'AS()'.</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="copy-to/csv">
+ <test-case FilePath="copy-to/csv">
+ <compilation-unit name="simple-csv">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">simple-csv</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/csv">
+ <compilation-unit name="type-mismatch">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">type-mismatch</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/csv">
+ <compilation-unit name="delimiter">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">delimiter</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="copy-to/csv">
+ <compilation-unit name="header">
+ <placeholder name="adapter" value="HDFS" />
+ <placeholder name="pathprefix" value='"/playground", ' />
+ <placeholder name="path_prefix" value="/playground/" />
+ <placeholder name="additionalProperties" value="" />
+ <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+ <output-dir compare="Text">header</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- TODO: This test is failing: ASTERIXDB-3529-->
+ <!-- <test-case FilePath="copy-to/csv">-->
+ <!-- <compilation-unit name="quote-escape">-->
+ <!-- <placeholder name="adapter" value="HDFS" />-->
+ <!-- <placeholder name="pathprefix" value='"/playground", ' />-->
+ <!-- <placeholder name="path_prefix" value="/playground/" />-->
+ <!-- <placeholder name="additionalProperties" value="" />-->
+ <!-- <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />-->
+ <!-- <output-dir compare="Text">quote-escape</output-dir>-->
+ <!-- </compilation-unit>-->
+ <!-- </test-case>-->
</test-group>
</test-suite>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index a27e325..135b93f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -22,8 +22,10 @@
import static org.apache.asterix.external.util.ExternalDataConstants.FORMAT_PARQUET;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+import java.io.FileNotFoundException;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -33,6 +35,7 @@
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.external.IExternalFilterEvaluator;
import org.apache.asterix.common.external.IExternalFilterEvaluatorFactory;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.api.IExternalDataRuntimeContext;
@@ -40,6 +43,7 @@
import org.apache.asterix.external.api.IRecordReader;
import org.apache.asterix.external.api.IRecordReaderFactory;
import org.apache.asterix.external.input.filter.embedder.IExternalFilterValueEmbedder;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetFileRecordReader;
import org.apache.asterix.external.input.record.reader.stream.StreamRecordReader;
@@ -47,8 +51,14 @@
import org.apache.asterix.external.provider.StreamRecordReaderProvider;
import org.apache.asterix.external.provider.context.ExternalStreamRuntimeDataContext;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataPrefix;
import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
@@ -62,15 +72,20 @@
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.hdfs.dataflow.ConfFactory;
import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
import org.apache.hyracks.hdfs.scheduler.Scheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IExternalDataSourceFactory {
private static final long serialVersionUID = 1L;
- private static final List<String> recordReaderNames = Collections.singletonList("hdfs");
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_HDFS);
+ private static final Logger LOGGER = LogManager.getLogger();
protected transient AlgebricksAbsolutePartitionConstraint clusterLocations;
protected transient IServiceContext serviceCtx;
@@ -108,9 +123,48 @@
} catch (IOException ex) {
throw HyracksDataException.create(ex);
}
+ if (!configuration.containsKey(ExternalDataConstants.KEY_PATH)) {
+ extractRequiredFiles(serviceCtx, configuration, warningCollector, filterEvaluatorFactory, hdfsConf);
+ }
configureHdfsConf(hdfsConf, configuration);
}
+ private void extractRequiredFiles(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector, IExternalFilterEvaluatorFactory filterEvaluatorFactory,
+ JobConf hdfsConf) throws HyracksDataException, AlgebricksException {
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher =
+ ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+
+ IExternalFilterEvaluator evaluator = filterEvaluatorFactory.create(serviceCtx, warningCollector);
+ ExternalDataPrefix externalDataPrefix = new ExternalDataPrefix(configuration);
+ configuration.put(ExternalDataPrefix.PREFIX_ROOT_FIELD_NAME, externalDataPrefix.getRoot());
+ try (FileSystem fs = ugi == null ? FileSystem.get(hdfsConf)
+ : ugi.doAs((PrivilegedExceptionAction<FileSystem>) () -> FileSystem.get(hdfsConf))) {
+ List<Path> reqFiles = new ArrayList<>();
+ RemoteIterator<LocatedFileStatus> files =
+ fs.listFiles(new Path(configuration.get(ExternalDataPrefix.PREFIX_ROOT_FIELD_NAME)), true);
+ while (files.hasNext()) {
+ LocatedFileStatus file = files.next();
+ if (ExternalDataUtils.evaluate(file.getPath().toUri().getPath(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), externalDataPrefix, evaluator, warningCollector)) {
+ reqFiles.add(file.getPath());
+ }
+ }
+ if (reqFiles.isEmpty()) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES));
+ }
+ HDFSUtils.setInputDir(hdfsConf, "");
+ } else {
+ FileInputFormat.setInputPaths(hdfsConf, reqFiles.toArray(new Path[0]));
+ }
+ } catch (FileNotFoundException ex) {
+ throw CompilationException.create(ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ } catch (InterruptedException | IOException ex) {
+ throw HyracksDataException.create(ex);
+ }
+ }
+
protected JobConf prepareHDFSConf(IServiceContext serviceCtx, Map<String, String> configuration,
IExternalFilterEvaluatorFactory filterEvaluatorFactory) throws HyracksDataException {
this.serviceCtx = serviceCtx;
@@ -127,13 +181,8 @@
confFactory = new ConfFactory(conf);
clusterLocations = getPartitionConstraint();
int numPartitions = clusterLocations.getLocations().length;
- InputSplit[] configInputSplits;
- if (credentials != null) {
- configInputSplits =
- ugi.doAs((PrivilegedExceptionAction<InputSplit[]>) () -> getInputSplits(conf, numPartitions));
- } else {
- configInputSplits = getInputSplits(conf, numPartitions);
- }
+ InputSplit[] configInputSplits = ugi == null ? getInputSplits(conf, numPartitions)
+ : ugi.doAs((PrivilegedExceptionAction<InputSplit[]>) () -> getInputSplits(conf, numPartitions));
readSchedule = hdfsScheduler.getLocationConstraints(configInputSplits);
inputSplitsFactory = new InputSplitsFactory(configInputSplits);
read = new boolean[readSchedule.length];
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
index 6d72fb5..6f338c4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
@@ -36,18 +36,13 @@
super(read, inputSplits, readSchedule, nodeName, conf, ugi);
}
- @SuppressWarnings("unchecked")
@Override
protected RecordReader<K, V> getRecordReader(int splitIndex) throws IOException {
- if (ugi != null) {
- try {
- reader = ugi.doAs((PrivilegedExceptionAction<RecordReader<K, V>>) () -> (RecordReader<K, V>) inputFormat
- .getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL));
- } catch (InterruptedException ex) {
- throw HyracksDataException.create(ex);
- }
- } else {
- reader = (RecordReader<K, V>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+ try {
+ reader = ugi == null ? getReader(splitIndex)
+ : ugi.doAs((PrivilegedExceptionAction<RecordReader<K, V>>) () -> getReader(splitIndex));
+ } catch (InterruptedException ex) {
+ throw HyracksDataException.create(ex);
}
if (key == null) {
key = reader.createKey();
@@ -56,6 +51,11 @@
return reader;
}
+ @SuppressWarnings("unchecked")
+ private RecordReader<K, V> getReader(int splitIndex) throws IOException {
+ return (RecordReader<K, V>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+ }
+
@Override
protected boolean onNextInputSplit() {
return false;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
index a1f86a7..c11885a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
@@ -64,15 +64,8 @@
@Override
protected RecordReader<Void, V> getRecordReader(int splitIndex) throws IOException {
try {
- ParquetRecordReaderWrapper readerWrapper;
- if (ugi != null) {
- readerWrapper = ugi
- .doAs((PrivilegedExceptionAction<ParquetRecordReaderWrapper>) () -> (ParquetRecordReaderWrapper) inputFormat
- .getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL));
- } else {
- readerWrapper = (ParquetRecordReaderWrapper) inputFormat.getRecordReader(inputSplits[splitIndex], conf,
- Reporter.NULL);
- }
+ ParquetRecordReaderWrapper readerWrapper = ugi == null ? getReader(splitIndex)
+ : ugi.doAs((PrivilegedExceptionAction<ParquetRecordReaderWrapper>) () -> getReader(splitIndex));
reader = (RecordReader<Void, V>) readerWrapper;
} catch (AsterixParquetRuntimeException e) {
throw e.getHyracksDataException();
@@ -84,4 +77,8 @@
}
return reader;
}
+
+ private ParquetRecordReaderWrapper getReader(int splitIndex) throws IOException {
+ return (ParquetRecordReaderWrapper) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java
index b81d815..d508336 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java
@@ -168,19 +168,12 @@
return false;
}
- @SuppressWarnings("unchecked")
private RecordReader<Object, Text> getRecordReader(int splitIndex) throws IOException {
- if (ugi != null) {
- try {
- reader = ugi
- .doAs((PrivilegedExceptionAction<RecordReader<Object, Text>>) () -> (RecordReader<Object, Text>) inputFormat
- .getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL));
- } catch (InterruptedException ex) {
- throw HyracksDataException.create(ex);
- }
- } else {
- reader = (RecordReader<Object, Text>) inputFormat.getRecordReader(inputSplits[splitIndex], conf,
- Reporter.NULL);
+ try {
+ reader = ugi == null ? getReader(splitIndex)
+ : ugi.doAs((PrivilegedExceptionAction<RecordReader<Object, Text>>) () -> getReader(splitIndex));
+ } catch (InterruptedException ex) {
+ throw HyracksDataException.create(ex);
}
if (key == null) {
@@ -189,4 +182,9 @@
}
return reader;
}
+
+ @SuppressWarnings("unchecked")
+ private RecordReader<Object, Text> getReader(int splitIndex) throws IOException {
+ return (RecordReader<Object, Text>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index 202e131..e758f64 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -162,6 +162,12 @@
public static final String KEY_ADAPTER_NAME_GCS = "GCS";
public static final String KEY_ADAPTER_NAME_HDFS = "HDFS";
+ public static final Set<String> EXTERNAL_READ_ADAPTERS = Set.of(KEY_ADAPTER_NAME_TWITTER_PUSH,
+ KEY_ADAPTER_NAME_PUSH_TWITTER, KEY_ADAPTER_NAME_TWITTER_PULL, KEY_ADAPTER_NAME_PULL_TWITTER,
+ KEY_ADAPTER_NAME_TWITTER_USER_STREAM, KEY_ADAPTER_NAME_LOCALFS, KEY_ADAPTER_NAME_SOCKET,
+ KEY_ADAPTER_NAME_HTTP, KEY_ADAPTER_NAME_AWS_S3, KEY_ADAPTER_NAME_AZURE_BLOB,
+ KEY_ADAPTER_NAME_AZURE_DATA_LAKE, KEY_ADAPTER_NAME_GCS, KEY_ADAPTER_NAME_HDFS);
+
/**
* HDFS class names
*/
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java
index 0ed0622..3d5865a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java
@@ -75,7 +75,8 @@
public ExternalDataPrefix(Map<String, String> configuration) throws AlgebricksException {
String prefix = ExternalDataUtils.getDefinitionOrPath(configuration);
- this.original = prefix != null ? prefix : "";
+ this.original = HDFSUtils.isSourceTypeHdfs(configuration) ? HDFSUtils.updateRootPath(prefix, true)
+ : (prefix != null ? prefix : "");
this.endsWithSlash = this.original.endsWith("/");
protocolContainerPair = ExternalDataUtils.getProtocolContainerPair(configuration);
segments = extractPrefixSegments(original);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 950f3e2..8c72a8c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -27,7 +27,6 @@
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_INCLUDE;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_PATH;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_READER;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureBlobProperties;
@@ -559,7 +558,8 @@
tableMetadataPath = S3Constants.HADOOP_S3_PROTOCOL + "://"
+ configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME) + '/'
+ configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
- } else if (configuration.get(ExternalDataConstants.KEY_READER).equals(ExternalDataConstants.READER_HDFS)) {
+ } else if (configuration.get(ExternalDataConstants.KEY_READER)
+ .equals(ExternalDataConstants.KEY_ADAPTER_NAME_HDFS)) {
conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI,
configuration.get(ExternalDataConstants.KEY_HDFS_URL));
tableMetadataPath = configuration.get(ExternalDataConstants.KEY_HDFS_URL) + '/' + tableMetadataPath;
@@ -1103,7 +1103,7 @@
String[] nodePathPair = path.trim().split("://");
protocol = nodePathPair[0];
break;
- case ExternalDataConstants.KEY_HDFS_URL:
+ case ExternalDataConstants.KEY_ADAPTER_NAME_HDFS:
protocol = ExternalDataConstants.KEY_HDFS_URL;
break;
default:
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index 2984954..75d68ba 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -233,7 +233,13 @@
ExternalDataConstants.CLASS_NAME_HDFS_FILESYSTEM);
conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, url);
}
- conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, configuration.get(ExternalDataConstants.KEY_PATH).trim());
+
+ // In case KEY_PATH is not being used, it means DEFINITION_FIELD_NAME is being used,
+ // and the property KEY_HADOOP_INPUT_DIR will be set in HDFSDatasourceFactory based on the provided definition.
+ String path = configuration.get(ExternalDataConstants.KEY_PATH);
+ if (path != null) {
+ HDFSUtils.setInputDir(conf, path.trim());
+ }
conf.setClassLoader(HDFSInputStream.class.getClassLoader());
conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_FORMAT, formatClassName);
@@ -397,6 +403,13 @@
}
}
+ public static String updateRootPath(String path, boolean updateNullPath) {
+ if (path != null && path.isEmpty()) {
+ return "/";
+ }
+ return path != null ? path : (updateNullPath ? "/" : null);
+ }
+
private static void configureParquet(Map<String, String> configuration, JobConf conf) {
//Parquet configurations
conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, ParquetReadSupport.class.getName());
@@ -528,13 +541,15 @@
return job.get(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, "").isEmpty();
}
+ public static void setInputDir(JobConf conf, String path) {
+ conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, path);
+ }
+
public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
IWarningCollector collector) throws CompilationException {
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
- if (configuration.get(ExternalDataConstants.KEY_PATH) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_PATH);
+ if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc,
+ ExternalDataConstants.KEY_INPUT_FORMAT);
}
if (Objects.equals(configuration.get(ExternalDataConstants.HADOOP_AUTHENTICATION),
@@ -562,6 +577,17 @@
}
}
+ // For validation purposes for external data prefixes
validateIncludeExclude(configuration);
+ try {
+ new ExternalDataPrefix(configuration);
+ } catch (AlgebricksException ex) {
+ throw new CompilationException(ErrorCode.FAILED_TO_CALCULATE_COMPUTED_FIELDS, ex);
+ }
+ }
+
+ public static boolean isSourceTypeHdfs(Map<String, String> configuration) {
+ return ExternalDataConstants.KEY_ADAPTER_NAME_HDFS
+ .equalsIgnoreCase(configuration.get(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE));
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java
index 18865d0..9203c8f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java
@@ -19,9 +19,12 @@
package org.apache.asterix.external.writer;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.util.HDFSUtils;
import org.apache.asterix.runtime.writer.IExternalFileWriter;
import org.apache.asterix.runtime.writer.IExternalPrinter;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -32,6 +35,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.hadoop.util.HiddenFileFilter;
public class HDFSExternalFileWriter implements IExternalFileWriter {
@@ -40,6 +44,7 @@
private final boolean partitionedPath;
private final SourceLocation pathSourceLocation;
private FSDataOutputStream outputStream = null;
+ private final List<Path> paths = new ArrayList<>();
HDFSExternalFileWriter(IExternalPrinter printer, FileSystem fs, boolean partitionedPath,
SourceLocation pathSourceLocation) {
@@ -57,6 +62,7 @@
@Override
public void validate(String directory) throws HyracksDataException {
if (partitionedPath) {
+ directory = HDFSUtils.updateRootPath(directory, true);
Path dirPath = new Path(directory);
try {
if (fs.exists(dirPath)) {
@@ -65,7 +71,7 @@
throw new RuntimeDataException(ErrorCode.DIRECTORY_IS_NOT_EMPTY, pathSourceLocation, directory);
}
if (fileStatus.isDirectory()) {
- FileStatus[] fileStatuses = fs.listStatus(dirPath);
+ FileStatus[] fileStatuses = fs.listStatus(dirPath, HiddenFileFilter.INSTANCE);
if (fileStatuses.length != 0) {
throw new RuntimeDataException(ErrorCode.DIRECTORY_IS_NOT_EMPTY, pathSourceLocation,
directory);
@@ -80,9 +86,11 @@
@Override
public boolean newFile(String directory, String fileName) throws HyracksDataException {
- Path path = new Path(directory, fileName);
+ directory = HDFSUtils.updateRootPath(directory, true);
+ Path path = new Path(directory, "." + fileName);
try {
outputStream = fs.create(path, false);
+ paths.add(path);
printer.newStream(outputStream);
} catch (FileAlreadyExistsException e) {
return false;
@@ -99,14 +107,25 @@
@Override
public void abort() throws HyracksDataException {
- if (outputStream != null) {
- outputStream.abort();
+ try {
+ printer.close();
+ for (Path path : paths) {
+ fs.delete(path, false);
+ }
+ } catch (IOException ex) {
+ throw HyracksDataException.create(ex);
}
- printer.close();
}
@Override
public void close() throws HyracksDataException {
printer.close();
+ try {
+ for (Path path : paths) {
+ fs.rename(path, new Path(path.getParent(), path.getName().substring(1)));
+ }
+ } catch (IOException ex) {
+ throw HyracksDataException.create(ex);
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java
index 761131f..dc20a89 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java
@@ -73,7 +73,7 @@
private HDFSExternalFileWriterFactory(ExternalFileWriterConfiguration externalConfig) {
configuration = externalConfig.getConfiguration();
- staticPath = externalConfig.getStaticPath();
+ staticPath = HDFSUtils.updateRootPath(externalConfig.getStaticPath(), false);
pathSourceLocation = externalConfig.getPathSourceLocation();
}
@@ -165,10 +165,10 @@
}
Path path = new Path(staticPath, "testFile");
try {
- FSDataOutputStream outputStream = fs.create(path);
- fs.deleteOnExit(path);
- outputStream.write(0);
- outputStream.close();
+ try (FSDataOutputStream outputStream = fs.create(path)) {
+ fs.deleteOnExit(path);
+ outputStream.write(0);
+ }
} catch (IOException ex) {
throw CompilationException.create(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
}