Merge branch 'gerrit/neo'
Change-Id: I7ba10b2a046866640ab3d646ea6b71aae399f436
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 06a6687..fed180b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -62,9 +62,12 @@
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
/**
* Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -157,7 +160,9 @@
nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
- unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
+ unnestMap.getGenerateCallBackProceedResultVar(),
+ isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()),
+ DefaultTupleProjectorFactory.INSTANCE);
IOperatorDescriptor opDesc = btreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
@@ -168,8 +173,12 @@
builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
}
- private boolean isPrimaryIndexPointSearch(ILogicalOperator op) {
- if (!isEqCondition || !isPrimaryIndex || !lowKeyVarList.equals(highKeyVarList)) {
+ /**
+ * Check whether we can use {@link LSMBTreeBatchPointSearchCursor} to perform point-lookups on the primary index
+ */
+ private boolean isPrimaryIndexPointSearch(ILogicalOperator op, PhysicalOptimizationConfig config) {
+ if (!config.isBatchLookupEnabled() || !isEqCondition || !isPrimaryIndex
+ || !lowKeyVarList.equals(highKeyVarList)) {
return false;
}
Index searchIndex = ((DataSourceIndex) idx).getIndex();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
index f625343..30c6623 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.compiler.provider;
+import java.util.Set;
+
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
import org.apache.asterix.algebra.base.ILangExtension;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -54,4 +56,9 @@
* @return the rule set factory of a language implementation
*/
IRuleSetFactory getRuleSetFactory();
+
+ /**
+ * @return all configurable parameters of a language implementation.
+ */
+ Set<String> getCompilerOptions();
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 2c18d41..43963dc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -18,14 +18,27 @@
*/
package org.apache.asterix.compiler.provider;
+import java.util.HashSet;
+import java.util.Set;
+
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.IRewriterFactory;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
import org.apache.asterix.lang.sqlpp.visitor.SqlppAstPrintVisitorFactory;
+import org.apache.asterix.optimizer.base.FuzzyUtils;
+import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
+import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
+import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
import org.apache.asterix.translator.SqlppExpressionToPlanTranslatorFactory;
public class SqlppCompilationProvider implements ILangCompilationProvider {
@@ -59,4 +72,24 @@
public IRuleSetFactory getRuleSetFactory() {
return new DefaultRuleSetFactory();
}
+
+ @Override
+ public Set<String> getCompilerOptions() {
+ return new HashSet<>(Set.of(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+ CompilerProperties.COMPILER_GROUPMEMORY_KEY, CompilerProperties.COMPILER_SORTMEMORY_KEY,
+ CompilerProperties.COMPILER_WINDOWMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+ CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+ CompilerProperties.COMPILER_SORT_SAMPLES_KEY, CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY,
+ CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
+ CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
+ CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+ CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+ FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+ StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+ FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+ SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+ DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+ SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
+ EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION));
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index b1d0b47..b8685d5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -26,7 +26,6 @@
import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionManager;
import org.apache.asterix.om.functions.IFunctionTypeInferer;
@@ -138,7 +137,7 @@
throws AlgebricksException {
IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
IFunctionDescriptor fd;
- if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+ if (expr.getFunctionInfo().isExternal()) {
// Expr is an external function
fd = ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context);
} else {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 10aedc7..5408915 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -48,7 +48,6 @@
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.constants.AsterixConstantValue;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -390,7 +389,7 @@
private boolean canConstantFold(ScalarFunctionCallExpression function) throws AlgebricksException {
// skip external functions because they're not available at compile time (on CC)
IFunctionInfo fi = function.getFunctionInfo();
- if (fi instanceof IExternalFunctionInfo) {
+ if (fi.isExternal()) {
return false;
}
// skip all functions that would produce records/arrays/multisets (derived types) in their open format
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 3a9d54d..ce86d24 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -54,7 +54,6 @@
import org.apache.asterix.dataflow.data.common.MergeAggregationExpressionFactory;
import org.apache.asterix.dataflow.data.common.MissableTypeComputer;
import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -65,25 +64,17 @@
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.statement.StartFeedStatement;
import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
-import org.apache.asterix.optimizer.base.FuzzyUtils;
-import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
-import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
-import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import org.apache.asterix.translator.ExecutionPlans;
import org.apache.asterix.translator.IRequestParameters;
import org.apache.asterix.translator.SessionConfig;
import org.apache.asterix.translator.SessionOutput;
-import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
import org.apache.asterix.utils.ResourceUtils;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -121,7 +112,6 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.ImmutableSet;
/**
* Provides helper methods for compilation of a query into a JobSpec and submission
@@ -133,29 +123,11 @@
public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
- // A white list of supported configurable parameters.
- private static final Set<String> CONFIGURABLE_PARAMETER_NAMES = ImmutableSet.of(
- CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
- CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_WINDOWMEMORY_KEY,
- CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
- CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
- CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, CompilerProperties.COMPILER_INDEXONLY_KEY,
- CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
- CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
- CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
- CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
- FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
- StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
- FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
- SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
- DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
- SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
- EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION);
-
private final IRewriterFactory rewriterFactory;
private final IAstPrintVisitorFactory astPrintVisitorFactory;
private final ILangExpressionToPlanTranslatorFactory translatorFactory;
private final IRuleSetFactory ruleSetFactory;
+ private final Set<String> configurableParameterNames;
private final ExecutionPlans executionPlans;
public APIFramework(ILangCompilationProvider compilationProvider) {
@@ -163,6 +135,7 @@
this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+ this.configurableParameterNames = compilationProvider.getCompilerOptions();
executionPlans = new ExecutionPlans();
}
@@ -502,10 +475,10 @@
}
// Validates if the query contains unsupported query parameters.
- private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
+ private Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
throws AlgebricksException {
for (String parameterName : config.keySet()) {
- if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)
+ if (!configurableParameterNames.contains(parameterName)
&& !parameterName.startsWith(PREFIX_INTERNAL_PARAMETERS)) {
throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
parameterName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 156b78a..48cf511 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -20,7 +20,7 @@
import static java.util.regex.Pattern.CASE_INSENSITIVE;
import static java.util.regex.Pattern.DOTALL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
import java.util.regex.Pattern;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
new file mode 100644
index 0000000..2608447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ with Batched point-lookups enabled.
+ */
+@RunWith(Parameterized.class)
+public class SqlppBatchPointLookupExecutionTest {
+ protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-batch-lookup.conf";
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ final TestExecutor testExecutor = new TestExecutor();
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ setNcEndpoints(testExecutor);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ @Parameters(name = "SqlppBatchPointLookupExecutionTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ return LangExecutionUtil.tests("only_batch_lookup.xml", "testsuite_sqlpp_batch_lookup.xml");
+ }
+
+ protected TestCaseContext tcCtx;
+
+ public SqlppBatchPointLookupExecutionTest(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LangExecutionUtil.test(tcCtx);
+ }
+
+ private static void setNcEndpoints(TestExecutor testExecutor) {
+ final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+ for (NodeControllerService nc : ncs) {
+ final String nodeId = nc.getId();
+ final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+ int apiPort = appCtx.getExternalProperties().getNcApiPort();
+ ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+ }
+ testExecutor.setNcEndPoints(ncEndPoints);
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
new file mode 100644
index 0000000..6e10481
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -0,0 +1,65 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements. See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership. The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License. You may obtain a copy of the License at
+;
+; http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied. See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=true
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="failed">
+ </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index c55c0bc..54793dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -10,6 +10,7 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 661daf3..06da0ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -10,6 +10,7 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 1f0e865..bef14c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -10,6 +10,7 @@
"active\.suspend\.timeout" : 3600,
"azure.request.timeout" : 120,
"compiler\.arrayindex" : true,
+ "compiler.batch.lookup" : false,
"compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
new file mode 100644
index 0000000..fd8b886
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
@@ -0,0 +1,862 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="batched-lookups">
+ <test-case FilePath="dml">
+ <compilation-unit name="compact-dataset-and-its-indexes">
+ <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-constant-merge-policy">
+ <output-dir compare="Text">using-constant-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-prefix-merge-policy">
+ <output-dir compare="Text">using-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-concurrent-merge-policy">
+ <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-correlated-prefix-merge-policy">
+ <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
+ <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="using-no-merge-policy">
+ <output-dir compare="Text">using-no-merge-policy</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-index">
+ <output-dir compare="Text">load-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-open">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
+ <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-01">
+ <output-dir compare="Text">fulltext-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-02">
+ <output-dir compare="Text">fulltext-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-08">
+ <output-dir compare="Text">fulltext-index-08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="fulltext-index-09">
+ <output-dir compare="Text">fulltext-index-09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="stopwords-full-text-filter-1">
+ <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-index-nested-loop-join">
+ <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_02">
+ <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_03">
+ <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-secondary-equi-join_04">
+ <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
+ <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
+ <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join" check-warnings="true">
+ <compilation-unit name="hints-indexnl-params">
+ <output-dir compare="Text">hints-indexnl-params</output-dir>
+ <expected-warn>ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 >>(8, idx_tenk2_1k_2k)<< Encountered <INTEGER_LITERAL> "8" at column 2. (in line 35, at column 21)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="rtree-spatial-intersect-point_05">
+ <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-03">
+ <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-01">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-02">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-03">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-composite-idxonly-04">
+ <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="btree-sidx-idxonly-01">
+ <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cust-index-age-nullable">
+ <output-dir compare="Text">cust-index-age-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="intersection-with-between">
+ <output-dir compare="Text">intersection-with-between</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
+ <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance">
+ <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-olist-jaccard">
+ <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="inverted-index-ulist-jaccard">
+ <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive-open">
+ <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey-open">
+ <output-dir compare="Text">orders-index-custkey-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="range-search-open">
+ <output-dir compare="Text">range-search-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="rtree-sidx-non-idxonly-02">
+ <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="dataset-with-meta">
+ <output-dir compare="Text">dataset-with-meta</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection" check-warnings="true">
+ <compilation-unit name="hints-skip-index">
+ <output-dir compare="Text">hints-skip-index</output-dir>
+ <expected-warn>ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 >>(13, idx_1k)<< Encountered <INTEGER_LITERAL> "13" at column 2. (in line 32, at column 19)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection" check-warnings="true">
+ <compilation-unit name="hints-use-index">
+ <output-dir compare="Text">hints-use-index</output-dir>
+ <expected-warn>ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>(18, idx_1k_2k)<< Encountered <INTEGER_LITERAL> "18" at column 2. (in line 33, at column 15)</expected-warn>
+ <expected-warn>ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>()<< Encountered ")" at column 2. (in line 33, at column 15)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="verify">
+ <output-dir compare="Text">verify</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-enforced/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-01">
+ <output-dir compare="Text">btree-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-02">
+ <output-dir compare="Text">btree-index-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-04">
+ <output-dir compare="Text">btree-index-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/correlated-index-selection">
+ <compilation-unit name="btree-index-01">
+ <output-dir compare="Text">btree-index-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="open-index-non-enforced/index-join">
+ <compilation-unit name="btree-equi-join-01">
+ <output-dir compare="Text">btree-equi-join-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-jaccard">
+ <output-dir compare="Text">ngram-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="ngram-jaccard-inline">
+ <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-non-enforced-equi-join">
+ <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-join">
+ <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+ <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-01">
+ <output-dir compare="Text">non-enforced-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-02">
+ <output-dir compare="Text">non-enforced-02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-open-index/index-selection">
+ <compilation-unit name="non-enforced-04">
+ <output-dir compare="Text">non-enforced-04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-composite-key-mixed-intervals">
+ <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="btree-index-rewrite-multiple">
+ <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="cust-index-age-nullable">
+ <output-dir compare="Text">cust-index-age-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-edit-distance">
+ <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-olist-jaccard">
+ <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="inverted-index-ulist-jaccard">
+ <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey">
+ <output-dir compare="Text">orders-index-custkey</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive">
+ <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-conjunctive-open">
+ <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="orders-index-custkey-open">
+ <output-dir compare="Text">orders-index-custkey-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="range-search">
+ <output-dir compare="Text">range-search</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="range-search-open">
+ <output-dir compare="Text">range-search-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index-optional">
+ <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="compact-dataset-and-its-indexes">
+ <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="delete-from-loaded-dataset-with-index">
+ <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+ <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-ngram-index">
+ <output-dir compare="Text">load-with-ngram-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-rtree-index">
+ <output-dir compare="Text">load-with-rtree-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="load-with-word-index">
+ <output-dir compare="Text">load-with-word-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="nested-index-dml">
+ <compilation-unit name="scan-delete-rtree-secondary-index">
+ <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="using-feed-new-index">
+ <output-dir compare="Text">using-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="cell-aggregation-with-filtering">
+ <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_join_01">
+ <output-dir compare="Text">outer_join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
+ <compilation-unit name="index_01">
+ <output-dir compare="Text">index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpcds">
+ <compilation-unit name="q45">
+ <output-dir compare="Text">q45</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q10_returned_item">
+ <output-dir compare="Text">q10_returned_item</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q10_returned_item_int64">
+ <output-dir compare="Text">q10_returned_item_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q12_shipping">
+ <output-dir compare="Text">q12_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q14_promotion_effect">
+ <output-dir compare="Text">q14_promotion_effect</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q15_top_supplier">
+ <output-dir compare="Text">q15_top_supplier</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q04_order_priority_with_nodegroup">
+ <output-dir compare="Text">q04_order_priority</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810">
+ <output-dir compare="Text">query-issue810</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810-2">
+ <output-dir compare="Text">query-issue810-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue810-3">
+ <output-dir compare="Text">query-issue810-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tpch-with-index">
+ <compilation-unit name="query-issue827-2">
+ <output-dir compare="Text">query-issue827-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
+ <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+ <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
+ <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
+ <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
+ <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-leftouterjoin">
+ <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
+ <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tinysocial">
+ <compilation-unit name="tinysocial-suite">
+ <output-dir compare="Text">tinysocial-suite</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="tinysocial">
+ <compilation-unit name="tinysocial-suite-open">
+ <output-dir compare="Text">tinysocial-suite-open</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-btree">
+ <output-dir compare="Text">load-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-btree-index-only">
+ <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-inverted-word">
+ <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="load-with-secondary-rtree">
+ <output-dir compare="Text">load-with-secondary-rtree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
+ <compilation-unit name="insert-with-correlated-secondary-btree">
+ <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
+ <compilation-unit name="union_opt_1">
+ <output-dir compare="Text">union_opt_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="upsert">
+ <compilation-unit name="primary-correlated-secondary-btree">
+ <output-dir compare="Text">primary-secondary-btree</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup">
+ <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup-select">
+ <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 5538a0a..9599435 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -103,7 +103,11 @@
COMPILER_ARRAYINDEX(
BOOLEAN,
AlgebricksConfig.ARRAY_INDEX_DEFAULT,
- "Enable/disable using array-indexes in queries");
+ "Enable/disable using array-indexes in queries"),
+ COMPILER_BATCH_LOOKUP(
+ BOOLEAN,
+ AlgebricksConfig.BATCH_LOOKUP_DEFAULT,
+ "Enable/disable batch point-lookups when running queries with secondary indexes");
private final IOptionType type;
private final Object defaultValue;
@@ -173,6 +177,8 @@
public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
+ public static final String COMPILER_BATCHED_LOOKUP_KEY = Option.COMPILER_BATCH_LOOKUP.ini();
+
public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
public CompilerProperties(PropertiesAccessor accessor) {
@@ -246,4 +252,8 @@
public int getExternalScanMemorySize() {
return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
}
+
+ public boolean isBatchLookup() {
+ return accessor.getBoolean(Option.COMPILER_BATCH_LOOKUP);
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index d498d69..d94fa04 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -77,6 +77,8 @@
int externalScanBufferSize = getExternalScanBufferSize(
(String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
compilerProperties.getExternalScanMemorySize(), sourceLoc);
+ boolean batchLookup = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_BATCHED_LOOKUP_KEY,
+ compilerProperties.isBatchLookup());
PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
physOptConf.setFrameSize(frameSize);
@@ -95,6 +97,7 @@
physOptConf.setMinMemoryAllocation(minMemoryAllocation);
physOptConf.setArrayIndexEnabled(arrayIndex);
physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+ physOptConf.setBatchLookup(batchLookup);
return physOptConf;
}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
index 94311f8..68c12f4 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
@@ -46,6 +46,7 @@
.options {
display: flex;
flex-flow: row;
+ overflow: auto;
}
.codemirror-container {
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
index f885edb..cb46a6a 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
@@ -538,7 +538,7 @@
this.queryString = '';
this.selected = 'Default';
} else {
- this.queryString = 'USE ' + this.selected + '; \n';
+ this.queryString = 'USE ' + this.selected + '; \n' + this.queryString;
}
this.editor.getDoc().setValue(this.queryString);
this.editor.execCommand('goDocEnd')
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
index a131353..4b6ecc5 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
@@ -97,7 +97,7 @@
[links]="edgesArr"
[nodes]="nodesArr"
[draggingEnabled]="false"
- [zoomSpeed]="0.025"
+ [zoomSpeed]="0.015"
[update$]="update$"
[layoutSettings]="{
orientation: planOrientation,
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
index 6c40a68..9601675 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
@@ -36,7 +36,6 @@
}
.panel {
- order: 2;
display: flex;
flex-flow: column;
justify-content: stretch;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
index 0fbf6f6..c5754762 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
@@ -30,7 +30,8 @@
<button *ngIf="isCSV" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export CSV file to Computer">EXPORT</button>
<button *ngIf="isCSV == false" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export JSON/JSONL file to Computer">EXPORT</button>
</span>
- <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pagedefaults.pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
+
+ <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
</mat-paginator>
</div>
<div *ngIf='treeVisible' class='navi-data' class='navi-data'>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
index 5309991..66a6dd6 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
@@ -60,7 +60,8 @@
currentRange: any;
/* see 10 records as initial set */
pagedefaults: any = { pageIndex: 0, pageSize:10, lenght: 0};
- pageSizeOptions = [5, 10, 25, 100, 200];
+ pageSize = 10;
+ pageSizeOptions = [5, 10, 25, 100, 200, 300, 400];
viewMode = 'JSON';
showGoTop = false;
showGoBottom = false;
@@ -141,6 +142,9 @@
}
showResults(range, expanded) {
+ // update pagesize
+ this.pageSize = range.pageSize;
+
this.currentRange = range;
this.currentIndex = this.currentRange.pageIndex;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index f14af53..bbcf9cd 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -18,7 +18,6 @@
*/
package org.apache.asterix.external.input.record.reader.aws;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -32,7 +31,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -109,7 +108,7 @@
}
private boolean shouldRetry(String errorCode, int currentRetry) {
- return currentRetry < MAX_RETRIES && AwsS3.isRetryableError(errorCode);
+ return currentRetry < MAX_RETRIES && S3Utils.isRetryableError(errorCode);
}
@Override
@@ -134,7 +133,7 @@
private S3Client buildAwsS3Client(Map<String, String> configuration) throws HyracksDataException {
try {
- return ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
+ return S3Utils.buildAwsS3Client(configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 89ea39e..a241354 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -26,6 +26,7 @@
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -53,8 +54,7 @@
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
//Get a list of S3 objects
- List<S3Object> filesOnly =
- ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
index 803e657..ff93a46 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
@@ -28,6 +28,8 @@
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Constants;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
@@ -52,7 +54,7 @@
//Configure Hadoop S3 input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
int numberOfPartitions = getPartitionConstraint().getLocations().length;
- ExternalDataUtils.AwsS3.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
+ S3Utils.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
configureHdfsConf(conf, configuration);
}
@@ -89,8 +91,7 @@
throws CompilationException {
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<S3Object> filesOnly =
- ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
if (!filesOnly.isEmpty()) {
@@ -105,7 +106,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, S3Object file) {
- builder.append(ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL);
+ builder.append(S3Constants.HADOOP_S3_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('/');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
index cdb3834..bbfece2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.azure.blob;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -31,7 +32,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -86,7 +86,7 @@
private BlobServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
throws HyracksDataException {
try {
- return ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+ return buildAzureBlobClient(appCtx, configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
index 064b319..55c0521 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.external.input.record.reader.azure.blob;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@@ -57,9 +60,9 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
- List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+ List<BlobItem> filesOnly =
+ listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
index e34d188..7a95222 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.azure.datalake;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
@@ -31,7 +32,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -86,7 +86,7 @@
private DataLakeServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
throws HyracksDataException {
try {
- return ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+ return buildAzureDatalakeClient(appCtx, configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
index e9f8d4c..929cb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.asterix.external.input.record.reader.azure.datalake;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
import java.util.Comparator;
import java.util.List;
import java.util.Map;
@@ -57,9 +60,9 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- DataLakeServiceClient client = ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
- List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(client, configuration,
- includeExcludeMatcher, warningCollector);
+ DataLakeServiceClient client = buildAzureDatalakeClient(appCtx, configuration);
+ List<PathItem> filesOnly =
+ listDatalakePathItems(client, configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
index c2251df..e08013c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.external.input.record.reader.azure.parquet;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -47,7 +52,7 @@
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+ BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
//Get endpoint
String endPoint = extractEndPoint(blobServiceClient.getAccountUrl());
//Get path
@@ -57,7 +62,7 @@
//Configure Hadoop Azure input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
- ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureAzureHdfsJobConf(conf, configuration, endPoint);
configureHdfsConf(conf, configuration);
}
@@ -94,8 +99,8 @@
private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
BlobServiceClient blobServiceClient, String endPoint) throws CompilationException {
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ List<BlobItem> filesOnly =
+ listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -118,7 +123,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, String endPoint, BlobItem file) {
- builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL);
+ builder.append(HADOOP_AZURE_BLOB_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
index db87868..c98fc8b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
@@ -18,6 +18,11 @@
*/
package org.apache.asterix.external.input.record.reader.azure.parquet;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_DATALAKE_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -47,8 +52,7 @@
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
- DataLakeServiceClient dataLakeServiceClient =
- ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+ DataLakeServiceClient dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
//Get endpoint
String endPoint = extractEndPoint(dataLakeServiceClient.getAccountUrl());
@@ -61,7 +65,7 @@
//Configure Hadoop Azure input splits
JobConf conf = createHdfsConf(serviceCtx, configuration);
- ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureAzureHdfsJobConf(conf, configuration, endPoint);
configureHdfsConf(conf, configuration);
}
@@ -98,8 +102,8 @@
private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
DataLakeServiceClient dataLakeServiceClient, String endPoint) throws CompilationException {
IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(dataLakeServiceClient, configuration,
- includeExcludeMatcher, warningCollector);
+ List<PathItem> filesOnly =
+ listDatalakePathItems(dataLakeServiceClient, configuration, includeExcludeMatcher, warningCollector);
StringBuilder builder = new StringBuilder();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -122,7 +126,7 @@
}
private static void appendFileURI(StringBuilder builder, String container, String endPoint, PathItem file) {
- builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_DATALAKE_PROTOCOL);
+ builder.append(HADOOP_AZURE_DATALAKE_PROTOCOL);
builder.append("://");
builder.append(container);
builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
index 652fa3e..007e8be 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -32,7 +32,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.CleanupUtils;
@@ -134,7 +134,7 @@
private Storage buildClient(Map<String, String> configuration) throws HyracksDataException {
try {
- return ExternalDataUtils.GCS.buildClient(configuration);
+ return GCSUtils.buildClient(configuration);
} catch (CompilationException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
index 0e7ea90..1bc51f2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
@@ -19,6 +19,7 @@
package org.apache.asterix.external.input.record.reader.gcs;
import static org.apache.asterix.external.util.ExternalDataUtils.getIncludeExcludeMatchers;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.buildClient;
import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.util.ArrayList;
@@ -67,7 +68,7 @@
// Prepare to retrieve the objects
List<Blob> filesOnly = new ArrayList<>();
String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- Storage gcs = ExternalDataUtils.GCS.buildClient(configuration);
+ Storage gcs = buildClient(configuration);
Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
Page<Blob> items;
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 f0b9c90..429706e 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
@@ -333,107 +333,4 @@
*/
public static final Set<String> VALID_TIME_ZONES = Set.of(TimeZone.getAvailableIDs());
}
-
- public static class AwsS3 {
- private AwsS3() {
- throw new AssertionError("do not instantiate");
- }
-
- public static final String REGION_FIELD_NAME = "region";
- public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
- public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
- public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
- public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
-
- // AWS S3 specific error codes
- public static final String ERROR_INTERNAL_ERROR = "InternalError";
- public static final String ERROR_SLOW_DOWN = "SlowDown";
- public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
-
- public static boolean isRetryableError(String errorCode) {
- return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
- }
-
- /*
- * Hadoop-AWS
- * AWS connectors for s3 and s3n are deprecated.
- */
- public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
- public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
- public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
- public static final String HADOOP_REGION = "fs.s3a.region";
- public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
-
- /*
- * Internal configurations
- */
- //Allows accessing directories as file system path
- public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
- //The number of maximum HTTP connections in connection pool
- public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
- //S3 used protocol
- public static final String HADOOP_S3_PROTOCOL = "s3a";
-
- //Hadoop credentials provider key
- public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
- //Anonymous credential provider
- public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
- //Temporary credential provider
- public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
-
- }
-
- /*
- * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
- * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
- */
- public static class Azure {
- private Azure() {
- throw new AssertionError("do not instantiate");
- }
-
- /*
- * Asterix Configuration Keys
- */
- public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
- public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
- public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
- public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
- public static final String TENANT_ID_FIELD_NAME = "tenantId";
- public static final String CLIENT_ID_FIELD_NAME = "clientId";
- public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
- public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
- public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
- public static final String ENDPOINT_FIELD_NAME = "endpoint";
-
- // Specific Azure data lake property
- /*
- The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
- storage/myData/personal/file1.json
- storage/myData/personal/file2.json
- storage/myData/file3.json
- If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
- is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
- is file1.json, file2.json and file3.json.
- */
- public static final String RECURSIVE_FIELD_NAME = "recursive";
-
- /*
- * Hadoop-Azure
- */
- //Used when accountName and accessKey are provided
- public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
- //Used when a connectionString is provided
- public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
- public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
- public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
- }
-
- public static class GCS {
- private GCS() {
- throw new AssertionError("do not instantiate");
- }
-
- public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
- }
}
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 8e38eed..702ef42 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
@@ -18,43 +18,9 @@
*/
package org.apache.asterix.external.util;
-import static com.google.cloud.storage.Storage.BlobListOption;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ANONYMOUS_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_CREDENTIAL_PROVIDER_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_PATH_STYLE_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_CONNECTION_POOL_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SECRET_ACCESS_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SESSION_TOKEN;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_TEMP_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_NAME_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_SECRET_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ENDPOINT_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_ACCOUNT_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_SAS;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.MANAGED_IDENTITY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
@@ -65,18 +31,14 @@
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
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;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureDataLakeProperties;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.validateProperties;
import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
-import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.URI;
-import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
@@ -85,7 +47,6 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
-import java.util.function.BiPredicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
@@ -106,20 +67,17 @@
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.library.JavaLibrary;
import org.apache.asterix.external.util.ExternalDataConstants.ParquetOptions;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.asterix.runtime.projection.DataProjectionInfo;
import org.apache.asterix.runtime.projection.FunctionCallInformation;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.dataflow.common.data.parsers.BooleanParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
@@ -129,46 +87,6 @@
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.util.StorageUtil;
-import com.azure.core.credential.AzureSasCredential;
-import com.azure.core.http.rest.PagedIterable;
-import com.azure.identity.ClientCertificateCredentialBuilder;
-import com.azure.identity.ClientSecretCredentialBuilder;
-import com.azure.identity.ManagedIdentityCredentialBuilder;
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.BlobServiceClientBuilder;
-import com.azure.storage.blob.models.BlobItem;
-import com.azure.storage.blob.models.ListBlobsOptions;
-import com.azure.storage.common.StorageSharedKeyCredential;
-import com.azure.storage.common.policy.RequestRetryOptions;
-import com.azure.storage.file.datalake.DataLakeFileSystemClient;
-import com.azure.storage.file.datalake.DataLakeServiceClient;
-import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
-import com.azure.storage.file.datalake.models.ListPathsOptions;
-import com.azure.storage.file.datalake.models.PathItem;
-import com.google.api.gax.paging.Page;
-import com.google.auth.oauth2.ServiceAccountCredentials;
-import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
-import com.google.cloud.storage.StorageOptions;
-
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
-import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.S3ClientBuilder;
-import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
-import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
-import software.amazon.awssdk.services.s3.model.S3Exception;
-import software.amazon.awssdk.services.s3.model.S3Object;
-import software.amazon.awssdk.services.s3.model.S3Response;
-
public class ExternalDataUtils {
private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
private static final int DEFAULT_MAX_ARGUMENT_SZ = 1024 * 1024;
@@ -604,16 +522,16 @@
switch (type) {
case ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3:
- AwsS3.validateProperties(configuration, srcLoc, collector);
+ S3Utils.validateProperties(configuration, srcLoc, collector);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB:
- Azure.validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
+ validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE:
- Azure.validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
+ validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
break;
case KEY_ADAPTER_NAME_GCS:
- GCS.validateProperties(configuration, srcLoc, collector);
+ validateProperties(configuration, srcLoc, collector);
break;
default:
// Nothing needs to be done
@@ -844,7 +762,7 @@
}
}
- private static boolean isParquetFormat(Map<String, String> properties) {
+ public static boolean isParquetFormat(Map<String, String> properties) {
String inputFormat = properties.get(ExternalDataConstants.KEY_INPUT_FORMAT);
return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(inputFormat)
|| ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)
@@ -893,1008 +811,6 @@
return encoder.encodeToString(byteArrayOutputStream.toByteArray());
}
- public static class AwsS3 {
- private AwsS3() {
- throw new AssertionError("do not instantiate");
- }
-
- /**
- * Builds the S3 client using the provided configuration
- *
- * @param configuration properties
- * @return S3 client
- * @throws CompilationException CompilationException
- */
- public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
- // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
- String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
- String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
- String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
- String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
- S3ClientBuilder builder = S3Client.builder();
-
- // Credentials
- AwsCredentialsProvider credentialsProvider;
-
- // No auth required
- if (accessKeyId == null) {
- credentialsProvider = AnonymousCredentialsProvider.create();
- } else {
- // auth required, check for temporary or permanent credentials
- if (sessionToken != null) {
- credentialsProvider = StaticCredentialsProvider
- .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
- } else {
- credentialsProvider =
- StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
- }
- }
-
- builder.credentialsProvider(credentialsProvider);
-
- // Validate the region
- List<Region> regions = S3Client.serviceMetadata().regions();
- Optional<Region> selectedRegion =
- regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
-
- if (selectedRegion.isEmpty()) {
- throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
- }
- builder.region(selectedRegion.get());
-
- // Validate the service endpoint if present
- if (serviceEndpoint != null) {
- try {
- URI uri = new URI(serviceEndpoint);
- try {
- builder.endpointOverride(uri);
- } catch (NullPointerException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (URISyntaxException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
- String.format("Invalid service endpoint %s", serviceEndpoint));
- }
- }
-
- return builder.build();
- }
-
- /**
- * Builds the S3 client using the provided configuration
- *
- * @param configuration properties
- * @param numberOfPartitions number of partitions in the cluster
- */
- public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
- int numberOfPartitions) {
- String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
- String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
- String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
- //Disable caching S3 FileSystem
- HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
-
- /*
- * Authentication Methods:
- * 1- Anonymous: no accessKeyId and no secretAccessKey
- * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
- * 3- Private: has to provide accessKeyId and secretAccessKey
- */
- if (accessKeyId == null) {
- //Tells hadoop-aws it is an anonymous access
- conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
- } else {
- conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
- conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
- if (sessionToken != null) {
- conf.set(HADOOP_SESSION_TOKEN, sessionToken);
- //Tells hadoop-aws it is a temporary access
- conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
- }
- }
-
- /*
- * This is to allow S3 definition to have path-style form. Should always be true to match the current
- * way we access files in S3
- */
- conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
-
- /*
- * Set the size of S3 connection pool to be the number of partitions
- */
- conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
-
- if (serviceEndpoint != null) {
- // Validation of the URL should be done at hadoop-aws level
- conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, serviceEndpoint);
- } else {
- //Region is ignored and buckets could be found by the central endpoint
- conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
- String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
- String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
- if (accessKeyId == null || secretAccessKey == null) {
- // If one is passed, the other is required
- if (accessKeyId != null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
- ACCESS_KEY_ID_FIELD_NAME);
- } else if (secretAccessKey != null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
- SECRET_ACCESS_KEY_FIELD_NAME);
- }
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- S3Client s3Client = buildAwsS3Client(configuration);
- S3Response response;
- boolean useOldApi = false;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- String prefix = getPrefix(configuration);
-
- try {
- response = isBucketEmpty(s3Client, container, prefix, false);
- } catch (S3Exception ex) {
- // Method not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
- useOldApi = true;
- response = isBucketEmpty(s3Client, container, prefix, true);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
- : ((ListObjectsV2Response) response).contents().isEmpty();
- if (isEmpty && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
-
- // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
- // ensure coverage, check if the result is successful as well and not only catch exceptions
- if (!response.sdkHttpResponse().isSuccessful()) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
- }
- }
-
- /**
- * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
- *
- * @param s3Client s3 client
- * @param container the container name
- * @param prefix Prefix to be used
- * @param useOldApi flag whether to use the old API or not
- * @return returns the S3 response
- */
- private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
- S3Response response;
- if (useOldApi) {
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
- listObjectsBuilder.prefix(prefix);
- response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
- } else {
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
- listObjectsBuilder.prefix(prefix);
- response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
- }
- return response;
- }
-
- /**
- * Returns the lists of S3 objects.
- *
- * @param configuration properties
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- public static List<S3Object> listS3Objects(Map<String, String> configuration,
- IncludeExcludeMatcher includeExcludeMatcher, IWarningCollector warningCollector)
- throws CompilationException {
- // Prepare to retrieve the objects
- List<S3Object> filesOnly;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- S3Client s3Client = buildAwsS3Client(configuration);
- String prefix = getPrefix(configuration);
-
- try {
- filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
- } catch (S3Exception ex) {
- // New API is not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode()
- .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
- filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the latest API to retrieve the objects from the storage.
- *
- * @param s3Client S3 client
- * @param container container name
- * @param prefix definition prefix
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsV2Response listObjectsResponse;
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
- listObjectsBuilder.prefix(prefix);
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
- } else {
- listObjectsResponse =
- s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextContinuationToken();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
- *
- * @param s3Client S3 client
- * @param container container name
- * @param prefix definition prefix
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsResponse listObjectsResponse;
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
- listObjectsBuilder.prefix(prefix);
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
- } else {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextMarker();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
- * a file if it does not end up with a "/" which is the separator in a folder structure.
- *
- * @param s3Objects List of returned objects
- */
- private static void collectAndFilterFiles(List<S3Object> s3Objects,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<S3Object> filesOnly) {
- for (S3Object object : s3Objects) {
- // skip folders
- if (object.key().endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, object.key())) {
- filesOnly.add(object);
- }
- }
- }
- }
-
- /*
- * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
- * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
- */
- public static class Azure {
- private Azure() {
- throw new AssertionError("do not instantiate");
- }
-
- /**
- * Builds the Azure storage account using the provided configuration
- *
- * @param configuration properties
- * @return client
- */
- public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx,
- Map<String, String> configuration) throws CompilationException {
- String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
- String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
- String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
- String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
- String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
- String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
- String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
- // Client builder
- BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
- int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
- RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
- builder.retryOptions(requestRetryOptions);
-
- // Endpoint is required
- if (endpoint == null) {
- throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
- }
- builder.endpoint(endpoint);
-
- // Shared Key
- if (accountName != null || accountKey != null) {
- if (accountName == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
- ACCOUNT_KEY_FIELD_NAME);
- }
-
- if (accountKey == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
- ACCOUNT_NAME_FIELD_NAME);
- }
-
- Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
- MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- ACCOUNT_KEY_FIELD_NAME);
- }
- StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
- builder.credential(credential);
- }
-
- // Shared access signature
- if (sharedAccessSignature != null) {
- Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
- builder.credential(credential);
- }
-
- // Managed Identity auth
- if (managedIdentityId != null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
- TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- MANAGED_IDENTITY_ID_FIELD_NAME);
- }
- builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
- }
-
- // Client secret & certificate auth
- if (clientId != null) {
- // Both (or neither) client secret and client secret were provided, only one is allowed
- if ((clientSecret == null) == (clientCertificate == null)) {
- if (clientSecret != null) {
- throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME);
- } else {
- throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
- }
- }
-
- // Tenant ID is required
- if (tenantId == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME);
- }
-
- // Client certificate password is not allowed if client secret is used
- if (clientCertificatePassword != null && clientSecret != null) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
- }
-
- // Use AD authentication
- if (clientSecret != null) {
- ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
- secret.clientId(clientId);
- secret.tenantId(tenantId);
- secret.clientSecret(clientSecret);
- builder.credential(secret.build());
- } else {
- // Certificate
- ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
- certificate.clientId(clientId);
- certificate.tenantId(tenantId);
- try {
- InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
- if (clientCertificatePassword == null) {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pemCertificate", InputStream.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent);
- } else {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
- }
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
- }
- builder.credential(certificate.build());
- }
- }
-
- // If client id is not present, ensure client secret, certificate, tenant id and client certificate
- // password are not present
- if (clientId == null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- }
-
- try {
- return builder.buildClient();
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Builds the Azure data lake storage account using the provided configuration
- *
- * @param configuration properties
- * @return client
- */
- public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
- Map<String, String> configuration) throws CompilationException {
- String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
- String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
- String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
- String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
- String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
- String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
- String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
- // Client builder
- DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
- int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
- RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
- builder.retryOptions(requestRetryOptions);
-
- // Endpoint is required
- if (endpoint == null) {
- throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
- }
- builder.endpoint(endpoint);
-
- // Shared Key
- if (accountName != null || accountKey != null) {
- if (accountName == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
- ACCOUNT_KEY_FIELD_NAME);
- }
-
- if (accountKey == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
- ACCOUNT_NAME_FIELD_NAME);
- }
-
- Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
- MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- ACCOUNT_KEY_FIELD_NAME);
- }
- StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
- builder.credential(credential);
- }
-
- // Shared access signature
- if (sharedAccessSignature != null) {
- Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
- builder.credential(credential);
- }
-
- // Managed Identity auth
- if (managedIdentityId != null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
- TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- MANAGED_IDENTITY_ID_FIELD_NAME);
- }
- builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
- }
-
- // Client secret & certificate auth
- if (clientId != null) {
- // Both (or neither) client secret and client secret were provided, only one is allowed
- if ((clientSecret == null) == (clientCertificate == null)) {
- if (clientSecret != null) {
- throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME);
- } else {
- throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
- CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
- }
- }
-
- // Tenant ID is required
- if (tenantId == null) {
- throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
- CLIENT_ID_FIELD_NAME);
- }
-
- // Client certificate password is not allowed if client secret is used
- if (clientCertificatePassword != null && clientSecret != null) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
- CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
- }
-
- // Use AD authentication
- if (clientSecret != null) {
- ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
- secret.clientId(clientId);
- secret.tenantId(tenantId);
- secret.clientSecret(clientSecret);
- builder.credential(secret.build());
- } else {
- // Certificate
- ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
- certificate.clientId(clientId);
- certificate.tenantId(tenantId);
- try {
- InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
- if (clientCertificatePassword == null) {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pemCertificate", InputStream.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent);
- } else {
- Method pemCertificate = ClientCertificateCredentialBuilder.class
- .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
- pemCertificate.setAccessible(true);
- pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
- }
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- builder.credential(certificate.build());
- }
- }
-
- // If client id is not present, ensure client secret, certificate, tenant id and client certificate
- // password are not present
- if (clientId == null) {
- Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
- CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
- if (provided.isPresent()) {
- throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
- SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- }
- }
-
- try {
- return builder.buildClient();
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient,
- Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
- IWarningCollector warningCollector) throws CompilationException {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<BlobItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- BlobContainerClient blobContainer;
- try {
- blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- // Collect the paths to files only
- collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- return filesOnly;
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
- for (BlobItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client,
- Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
- IWarningCollector warningCollector) throws CompilationException {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<PathItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- DataLakeFileSystemClient fileSystemClient;
- try {
- fileSystemClient = client.getFileSystemClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListPathsOptions listOptions = new ListPathsOptions();
- boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
- listOptions.setRecursive(recursive);
- listOptions.setPath(ExternalDataUtils.getPrefix(configuration, false));
- PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
-
- // Collect the paths to files only
- collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
-
- return filesOnly;
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
- BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
- for (PathItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- BlobServiceClient blobServiceClient;
- try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- blobServiceClient = buildAzureBlobClient(appCtx, configuration);
- BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- validateIncludeExclude(configuration);
-
- // Check if the bucket is present
- DataLakeServiceClient dataLakeServiceClient;
- try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
- DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListPathsOptions listPathsOptions = new ListPathsOptions();
- listPathsOptions.setPath(getPrefix(configuration));
- Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- /**
- * Builds the Azure Blob storage client using the provided configuration
- *
- * @param configuration properties
- * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
- * Blob storage</a>
- */
- public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
- String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-
- //Disable caching S3 FileSystem
- HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
-
- //Key for Hadoop configuration
- StringBuilder hadoopKey = new StringBuilder();
- //Value for Hadoop configuration
- String hadoopValue;
- if (accountKey != null || sharedAccessSignature != null) {
- if (accountKey != null) {
- hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
- //Set only the AccountKey
- hadoopValue = accountKey;
- } else {
- //Use SAS for Hadoop FS as connectionString is provided
- hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
- //Setting the container is required for SAS
- hadoopKey.append(container).append('.');
- //Set the connection string for SAS
- hadoopValue = sharedAccessSignature;
- }
- //Set the endPoint, which includes the AccountName
- hadoopKey.append(endPoint);
- //Tells Hadoop we are reading from Blob Storage
- conf.set(hadoopKey.toString(), hadoopValue);
- }
- }
- }
-
- public static class GCS {
- private GCS() {
- throw new AssertionError("do not instantiate");
-
- }
-
- //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
- //upon creating the external dataset
-
- /**
- * Builds the client using the provided configuration
- *
- * @param configuration properties
- * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
- * @throws CompilationException CompilationException
- */
- public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
- String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
-
- StorageOptions.Builder builder = StorageOptions.newBuilder();
-
- // Use credentials if available
- if (jsonCredentials != null) {
- try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
- builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
- } catch (IOException ex) {
- throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
-
- return builder.build().getService();
- }
-
- /**
- * Validate external dataset properties
- *
- * @param configuration properties
- * @throws CompilationException Compilation exception
- */
- public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
- IWarningCollector collector) throws CompilationException {
-
- // check if the format property is present
- if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
- throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
- }
-
- // parquet is not supported for google cloud storage
- if (isParquetFormat(configuration)) {
- throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT,
- configuration.get(KEY_FORMAT));
- }
-
- validateIncludeExclude(configuration);
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- try {
- BlobListOption limitOption = BlobListOption.pageSize(1);
- BlobListOption prefixOption = BlobListOption.prefix(getPrefix(configuration));
- Storage storage = buildClient(configuration);
- Page<Blob> items = storage.list(container, limitOption, prefixOption);
-
- if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
- Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- collector.warn(warning);
- }
- } catch (CompilationException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
- }
- }
- }
-
public static int roundUpToNearestFrameSize(int size, int framesize) {
return ((size / framesize) + 1) * framesize;
}
@@ -1911,7 +827,7 @@
return maxArgSz;
}
- private static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
+ public static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
return Arrays.stream(parameters).filter(field -> configuration.get(field) != null).findFirst();
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
new file mode 100644
index 0000000..e1c10ad
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+public class S3Constants {
+ private S3Constants() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static final String REGION_FIELD_NAME = "region";
+ public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
+ public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
+ public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
+ public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
+
+ // AWS S3 specific error codes
+ public static final String ERROR_INTERNAL_ERROR = "InternalError";
+ public static final String ERROR_SLOW_DOWN = "SlowDown";
+ public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
+
+ /*
+ * Hadoop-AWS
+ * AWS connectors for s3 and s3n are deprecated.
+ */
+ public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
+ public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
+ public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
+ public static final String HADOOP_REGION = "fs.s3a.region";
+ public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
+
+ /*
+ * Internal configurations
+ */
+ //Allows accessing directories as file system path
+ public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
+ //The number of maximum HTTP connections in connection pool
+ public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
+ //S3 used protocol
+ public static final String HADOOP_S3_PROTOCOL = "s3a";
+
+ //Hadoop credentials provider key
+ public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
+ //Anonymous credential provider
+ public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
+ //Temporary credential provider
+ public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
new file mode 100644
index 0000000..a88d59b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
@@ -0,0 +1,436 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.*;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.S3Response;
+
+public class S3Utils {
+ private S3Utils() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static boolean isRetryableError(String errorCode) {
+ return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
+ }
+
+ /**
+ * Builds the S3 client using the provided configuration
+ *
+ * @param configuration properties
+ * @return S3 client
+ * @throws CompilationException CompilationException
+ */
+ public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
+ // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+ String regionId = configuration.get(REGION_FIELD_NAME);
+ String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+ S3ClientBuilder builder = S3Client.builder();
+
+ // Credentials
+ AwsCredentialsProvider credentialsProvider;
+
+ // No auth required
+ if (accessKeyId == null) {
+ credentialsProvider = AnonymousCredentialsProvider.create();
+ } else {
+ // auth required, check for temporary or permanent credentials
+ if (sessionToken != null) {
+ credentialsProvider = StaticCredentialsProvider
+ .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+ } else {
+ credentialsProvider =
+ StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+ }
+ }
+
+ builder.credentialsProvider(credentialsProvider);
+
+ // Validate the region
+ List<Region> regions = S3Client.serviceMetadata().regions();
+ Optional<Region> selectedRegion = regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
+
+ if (selectedRegion.isEmpty()) {
+ throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
+ }
+ builder.region(selectedRegion.get());
+
+ // Validate the service endpoint if present
+ if (serviceEndpoint != null) {
+ try {
+ URI uri = new URI(serviceEndpoint);
+ try {
+ builder.endpointOverride(uri);
+ } catch (NullPointerException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (URISyntaxException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
+ String.format("Invalid service endpoint %s", serviceEndpoint));
+ }
+ }
+
+ return builder.build();
+ }
+
+ /**
+ * Builds the S3 client using the provided configuration
+ *
+ * @param configuration properties
+ * @param numberOfPartitions number of partitions in the cluster
+ */
+ public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
+ int numberOfPartitions) {
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+ String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+ //Disable caching S3 FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
+
+ /*
+ * Authentication Methods:
+ * 1- Anonymous: no accessKeyId and no secretAccessKey
+ * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
+ * 3- Private: has to provide accessKeyId and secretAccessKey
+ */
+ if (accessKeyId == null) {
+ //Tells hadoop-aws it is an anonymous access
+ conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
+ } else {
+ conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
+ conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
+ if (sessionToken != null) {
+ conf.set(HADOOP_SESSION_TOKEN, sessionToken);
+ //Tells hadoop-aws it is a temporary access
+ conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
+ }
+ }
+
+ /*
+ * This is to allow S3 definition to have path-style form. Should always be true to match the current
+ * way we access files in S3
+ */
+ conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
+
+ /*
+ * Set the size of S3 connection pool to be the number of partitions
+ */
+ conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
+
+ if (serviceEndpoint != null) {
+ // Validation of the URL should be done at hadoop-aws level
+ conf.set(HADOOP_SERVICE_END_POINT, serviceEndpoint);
+ } else {
+ //Region is ignored and buckets could be found by the central endpoint
+ conf.set(HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ if (accessKeyId == null || secretAccessKey == null) {
+ // If one is passed, the other is required
+ if (accessKeyId != null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+ ACCESS_KEY_ID_FIELD_NAME);
+ } else if (secretAccessKey != null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+ SECRET_ACCESS_KEY_FIELD_NAME);
+ }
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ S3Client s3Client = buildAwsS3Client(configuration);
+ S3Response response;
+ boolean useOldApi = false;
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ String prefix = getPrefix(configuration);
+
+ try {
+ response = isBucketEmpty(s3Client, container, prefix, false);
+ } catch (S3Exception ex) {
+ // Method not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+ useOldApi = true;
+ response = isBucketEmpty(s3Client, container, prefix, true);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
+ : ((ListObjectsV2Response) response).contents().isEmpty();
+ if (isEmpty && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+
+ // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
+ // ensure coverage, check if the result is successful as well and not only catch exceptions
+ if (!response.sdkHttpResponse().isSuccessful()) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
+ }
+ }
+
+ /**
+ * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
+ *
+ * @param s3Client s3 client
+ * @param container the container name
+ * @param prefix Prefix to be used
+ * @param useOldApi flag whether to use the old API or not
+ * @return returns the S3 response
+ */
+ private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
+ S3Response response;
+ if (useOldApi) {
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
+ listObjectsBuilder.prefix(prefix);
+ response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
+ } else {
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
+ listObjectsBuilder.prefix(prefix);
+ response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
+ }
+ return response;
+ }
+
+ /**
+ * Returns the lists of S3 objects.
+ *
+ * @param configuration properties
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ public static List<S3Object> listS3Objects(Map<String, String> configuration,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ // Prepare to retrieve the objects
+ List<S3Object> filesOnly;
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ S3Client s3Client = buildAwsS3Client(configuration);
+ String prefix = getPrefix(configuration);
+
+ try {
+ filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+ } catch (S3Exception ex) {
+ // New API is not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+ filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the latest API to retrieve the objects from the storage.
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param prefix definition prefix
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsV2Response listObjectsResponse;
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
+ listObjectsBuilder.prefix(prefix);
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextContinuationToken();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param prefix definition prefix
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsResponse listObjectsResponse;
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
+ listObjectsBuilder.prefix(prefix);
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextMarker();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+ * a file if it does not end up with a "/" which is the separator in a folder structure.
+ *
+ * @param s3Objects List of returned objects
+ */
+ private static void collectAndFilterFiles(List<S3Object> s3Objects, BiPredicate<List<Matcher>, String> predicate,
+ List<Matcher> matchers, List<S3Object> filesOnly) {
+ for (S3Object object : s3Objects) {
+ // skip folders
+ if (object.key().endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, object.key())) {
+ filesOnly.add(object);
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
new file mode 100644
index 0000000..9ade27b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.azure.blob_storage;
+
+/*
+ * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
+ * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
+ */
+public class AzureConstants {
+ private AzureConstants() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ /*
+ * Asterix Configuration Keys
+ */
+ public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
+ public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
+ public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
+ public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
+ public static final String TENANT_ID_FIELD_NAME = "tenantId";
+ public static final String CLIENT_ID_FIELD_NAME = "clientId";
+ public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
+ public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
+ public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
+ public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+ // Specific Azure data lake property
+ /*
+ The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
+ storage/myData/personal/file1.json
+ storage/myData/personal/file2.json
+ storage/myData/file3.json
+ If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
+ is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
+ is file1.json, file2.json and file3.json.
+ */
+ public static final String RECURSIVE_FIELD_NAME = "recursive";
+
+ /*
+ * Hadoop-Azure
+ */
+ //Used when accountName and accessKey are provided
+ public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
+ //Used when a connectionString is provided
+ public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
+ public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
+ public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
new file mode 100644
index 0000000..0dc9ad2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
@@ -0,0 +1,636 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.azure.blob_storage;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getFirstNotNull;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_NAME_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_SECRET_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_ACCOUNT_KEY;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_SAS;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.MANAGED_IDENTITY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.RECURSIVE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.TENANT_ID_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+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.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.identity.ClientCertificateCredentialBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.common.policy.RequestRetryOptions;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import com.azure.storage.file.datalake.models.PathItem;
+
+public class AzureUtils {
+ private AzureUtils() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ /**
+ * Builds the Azure storage account using the provided configuration
+ *
+ * @param configuration properties
+ * @return client
+ */
+ public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx, Map<String, String> configuration)
+ throws CompilationException {
+ String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+ String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+ String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+ String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+ String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+ String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ // Client builder
+ BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
+ int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+ RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+ builder.retryOptions(requestRetryOptions);
+
+ // Endpoint is required
+ if (endpoint == null) {
+ throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+ }
+ builder.endpoint(endpoint);
+
+ // Shared Key
+ if (accountName != null || accountKey != null) {
+ if (accountName == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+
+ if (accountKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+ ACCOUNT_NAME_FIELD_NAME);
+ }
+
+ Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+ MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+ StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+ builder.credential(credential);
+ }
+
+ // Shared access signature
+ if (sharedAccessSignature != null) {
+ Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+ builder.credential(credential);
+ }
+
+ // Managed Identity auth
+ if (managedIdentityId != null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ MANAGED_IDENTITY_ID_FIELD_NAME);
+ }
+ builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+ }
+
+ // Client secret & certificate auth
+ if (clientId != null) {
+ // Both (or neither) client secret and client secret were provided, only one is allowed
+ if ((clientSecret == null) == (clientCertificate == null)) {
+ if (clientSecret != null) {
+ throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME);
+ } else {
+ throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+ }
+ }
+
+ // Tenant ID is required
+ if (tenantId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME);
+ }
+
+ // Client certificate password is not allowed if client secret is used
+ if (clientCertificatePassword != null && clientSecret != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+ }
+
+ // Use AD authentication
+ if (clientSecret != null) {
+ ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+ secret.clientId(clientId);
+ secret.tenantId(tenantId);
+ secret.clientSecret(clientSecret);
+ builder.credential(secret.build());
+ } else {
+ // Certificate
+ ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+ certificate.clientId(clientId);
+ certificate.tenantId(tenantId);
+ try {
+ InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+ if (clientCertificatePassword == null) {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pemCertificate", InputStream.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent);
+ } else {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ }
+ builder.credential(certificate.build());
+ }
+ }
+
+ // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+ // password are not present
+ if (clientId == null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ }
+
+ try {
+ return builder.buildClient();
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Builds the Azure data lake storage account using the provided configuration
+ *
+ * @param configuration properties
+ * @return client
+ */
+ public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
+ Map<String, String> configuration) throws CompilationException {
+ String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+ String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+ String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+ String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+ String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+ String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ // Client builder
+ DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
+ int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+ RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+ builder.retryOptions(requestRetryOptions);
+
+ // Endpoint is required
+ if (endpoint == null) {
+ throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+ }
+ builder.endpoint(endpoint);
+
+ // Shared Key
+ if (accountName != null || accountKey != null) {
+ if (accountName == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+
+ if (accountKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+ ACCOUNT_NAME_FIELD_NAME);
+ }
+
+ Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+ MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+ StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+ builder.credential(credential);
+ }
+
+ // Shared access signature
+ if (sharedAccessSignature != null) {
+ Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+ builder.credential(credential);
+ }
+
+ // Managed Identity auth
+ if (managedIdentityId != null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ MANAGED_IDENTITY_ID_FIELD_NAME);
+ }
+ builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+ }
+
+ // Client secret & certificate auth
+ if (clientId != null) {
+ // Both (or neither) client secret and client secret were provided, only one is allowed
+ if ((clientSecret == null) == (clientCertificate == null)) {
+ if (clientSecret != null) {
+ throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME);
+ } else {
+ throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+ }
+ }
+
+ // Tenant ID is required
+ if (tenantId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME);
+ }
+
+ // Client certificate password is not allowed if client secret is used
+ if (clientCertificatePassword != null && clientSecret != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+ }
+
+ // Use AD authentication
+ if (clientSecret != null) {
+ ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+ secret.clientId(clientId);
+ secret.tenantId(tenantId);
+ secret.clientSecret(clientSecret);
+ builder.credential(secret.build());
+ } else {
+ // Certificate
+ ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+ certificate.clientId(clientId);
+ certificate.tenantId(tenantId);
+ try {
+ InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+ if (clientCertificatePassword == null) {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pemCertificate", InputStream.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent);
+ } else {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ builder.credential(certificate.build());
+ }
+ }
+
+ // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+ // password are not present
+ if (clientId == null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ }
+
+ try {
+ return builder.buildClient();
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient, Map<String, String> configuration,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ List<BlobItem> filesOnly = new ArrayList<>();
+
+ // Ensure the validity of include/exclude
+ validateIncludeExclude(configuration);
+
+ BlobContainerClient blobContainer;
+ try {
+ blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+ listBlobsOptions.setPrefix(getPrefix(configuration));
+ Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+ // Collect the paths to files only
+ collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Collects and filters the files only, and excludes any folders
+ *
+ * @param items storage items
+ * @param predicate predicate to test with for file filtration
+ * @param matchers include/exclude matchers to test against
+ * @param filesOnly List containing the files only (excluding folders)
+ */
+ private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
+ for (BlobItem item : items) {
+ String uri = item.getName();
+
+ // skip folders
+ if (uri.endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, uri)) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client, Map<String, String> configuration,
+ AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ List<PathItem> filesOnly = new ArrayList<>();
+
+ // Ensure the validity of include/exclude
+ validateIncludeExclude(configuration);
+
+ DataLakeFileSystemClient fileSystemClient;
+ try {
+ fileSystemClient = client.getFileSystemClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListPathsOptions listOptions = new ListPathsOptions();
+ boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
+ listOptions.setRecursive(recursive);
+ listOptions.setPath(getPrefix(configuration, false));
+ PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
+
+ // Collect the paths to files only
+ collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Collects and filters the files only, and excludes any folders
+ *
+ * @param items storage items
+ * @param predicate predicate to test with for file filtration
+ * @param matchers include/exclude matchers to test against
+ * @param filesOnly List containing the files only (excluding folders)
+ */
+ private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
+ for (PathItem item : items) {
+ String uri = item.getName();
+
+ // skip folders
+ if (uri.endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, uri)) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ BlobServiceClient blobServiceClient;
+ try {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+ BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+ listBlobsOptions.setPrefix(getPrefix(configuration));
+ Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+ if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ DataLakeServiceClient dataLakeServiceClient;
+ try {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
+ DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListPathsOptions listPathsOptions = new ListPathsOptions();
+ listPathsOptions.setPath(getPrefix(configuration));
+ Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
+
+ if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Builds the Azure Blob storage client using the provided configuration
+ *
+ * @param configuration properties
+ * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
+ * Blob storage</a>
+ */
+ public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+
+ //Disable caching S3 FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
+
+ //Key for Hadoop configuration
+ StringBuilder hadoopKey = new StringBuilder();
+ //Value for Hadoop configuration
+ String hadoopValue;
+ if (accountKey != null || sharedAccessSignature != null) {
+ if (accountKey != null) {
+ hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
+ //Set only the AccountKey
+ hadoopValue = accountKey;
+ } else {
+ //Use SAS for Hadoop FS as connectionString is provided
+ hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
+ //Setting the container is required for SAS
+ hadoopKey.append(container).append('.');
+ //Set the connection string for SAS
+ hadoopValue = sharedAccessSignature;
+ }
+ //Set the endPoint, which includes the AccountName
+ hadoopKey.append(endPoint);
+ //Tells Hadoop we are reading from Blob Storage
+ conf.set(hadoopKey.toString(), hadoopValue);
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
new file mode 100644
index 0000000..8a0be99
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+public class GCSConstants {
+ private GCSConstants() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
new file mode 100644
index 0000000..553733f
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_FORMAT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.isParquetFormat;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.JSON_CREDENTIALS_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.google.api.gax.paging.Page;
+import com.google.auth.oauth2.ServiceAccountCredentials;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.StorageOptions;
+
+public class GCSUtils {
+ private GCSUtils() {
+ throw new AssertionError("do not instantiate");
+
+ }
+
+ //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
+ //upon creating the external dataset
+
+ /**
+ * Builds the client using the provided configuration
+ *
+ * @param configuration properties
+ * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+ * @throws CompilationException CompilationException
+ */
+ public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
+ String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+
+ StorageOptions.Builder builder = StorageOptions.newBuilder();
+
+ // Use credentials if available
+ if (jsonCredentials != null) {
+ try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+ builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
+ } catch (IOException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ return builder.build().getService();
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ // parquet is not supported for google cloud storage
+ if (isParquetFormat(configuration)) {
+ throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT, configuration.get(KEY_FORMAT));
+ }
+
+ validateIncludeExclude(configuration);
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ try {
+ Storage.BlobListOption limitOption = Storage.BlobListOption.pageSize(1);
+ Storage.BlobListOption prefixOption = Storage.BlobListOption.prefix(getPrefix(configuration));
+ Storage storage = buildClient(configuration);
+ Page<Blob> items = storage.list(container, limitOption, prefixOption);
+
+ if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
index 90ea04b..91afbd8 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
@@ -18,8 +18,8 @@
*/
package org.apache.asterix.external.input.record.reader.awss3;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_INTERNAL_ERROR;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_SLOW_DOWN;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_INTERNAL_ERROR;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_SLOW_DOWN;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 557f17d..4352210 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -61,7 +61,7 @@
/**
* This rewriter is used to rewrite body expression of user defined functions and views
*/
-class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
+public class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
public SqlppFunctionBodyRewriter(IParserFactory parserFactory) {
super(parserFactory);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 1d82b19..14d1a6a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -53,6 +53,7 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
public class DatasetDataSource extends DataSource {
@@ -135,7 +136,7 @@
return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
- outputLimit, false, false);
+ outputLimit, false, false, DefaultTupleProjectorFactory.INSTANCE);
default:
throw new AlgebricksException("Unknown datasource type");
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 4cc83ab..1cdbbea 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -169,6 +169,7 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class MetadataProvider implements IMetadataProvider<DataSourceId, String> {
@@ -542,7 +543,8 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
- boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
+ boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch, ITupleProjectorFactory tupleProjectorFactory)
+ throws AlgebricksException {
boolean isSecondary = true;
Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), dataset.getDatasetName());
@@ -601,12 +603,13 @@
? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+ maxFilterFieldIndexes, tupleFilterFactory, outputLimit, tupleProjectorFactory)
: new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
- proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+ proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan,
+ tupleProjectorFactory);
} else {
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -859,7 +862,7 @@
*
* @param dataset
* @return Number of elements that will be used to create a bloom filter per
- * dataset per partition
+ * dataset per partition
* @throws AlgebricksException
*/
public long getCardinalityPerPartitionHint(Dataset dataset) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 5fd96a6..ce98008 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -174,7 +174,7 @@
if (fnKind != FunctionKind.SCALAR) {
return false;
}
- if (!(fnInfo instanceof IExternalFunctionInfo)) {
+ if (!fnInfo.isExternal()) {
return false;
}
ExternalFunctionLanguage language = ((IExternalFunctionInfo) fnInfo).getLanguage();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
index a3ec9c6..8a2cefa 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
@@ -101,4 +101,9 @@
public boolean getNullCall() {
return nullCall;
}
+
+ @Override
+ public boolean isExternal() {
+ return true;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
index 6e1ccef..34ed225 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
@@ -38,6 +38,11 @@
return true;
}
+ default boolean isExternal() {
+ // A function is not external by default.
+ return false;
+ }
+
/**
* @param args,
* the arguments.
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 5fc69b2..56e2e22 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -25,10 +25,10 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
- protected final Mutable<ILogicalExpression> expression;
protected final List<Object> variableTypes;
protected boolean propagateInput;
protected List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -42,7 +42,6 @@
public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
List<Object> variableTypes, boolean propagateInput) {
super(variables, expression);
- this.expression = expression;
this.variableTypes = variableTypes;
this.propagateInput = propagateInput;
this.propagateIndexFilter = false;
@@ -90,6 +89,18 @@
};
}
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = super.acceptExpressionTransform(visitor);
+
+ if (additionalFilteringExpressions != null) {
+ for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+ changed |= visitor.transform(filterExpr);
+ }
+ }
+ return changed;
+ }
+
public boolean propagatesInput() {
return propagateInput;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index bb18014..9f73113 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -57,7 +57,7 @@
}
public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
- Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
+ Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
super(variables, dataSource);
projectVars = new ArrayList<>();
this.selectCondition = selectCondition;
@@ -77,7 +77,13 @@
@Override
public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
- return false;
+ boolean changed = selectCondition != null && visitor.transform(selectCondition);
+ if (additionalFilteringExpressions != null) {
+ for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+ changed |= visitor.transform(filterExpr);
+ }
+ }
+ return changed;
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index c4bcc52..f8d07b8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
public class UnnestMapOperator extends AbstractUnnestMapOperator {
@@ -61,6 +62,12 @@
return visitor.visitUnnestMapOperator(this, arg);
}
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = super.acceptExpressionTransform(visitor);
+ return selectCondition != null && visitor.transform(selectCondition) || changed;
+ }
+
// When propagateInput is true,
// this operator propagates all input variables.
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
index 083e4d3..2bd78b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
@@ -43,7 +42,9 @@
@Override
public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ if (op2.getDeliveredPhysicalProperties() != null) {
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 86c6066..b626ae6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -37,4 +37,5 @@
public static final boolean ARRAY_INDEX_DEFAULT = true;
public static final int EXTERNAL_SCAN_BUFFER_SIZE =
StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
+ public static final boolean BATCH_LOOKUP_DEFAULT = false;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 9c9b21d..4d54a10 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -47,6 +47,7 @@
private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
private static final String ARRAY_INDEX = "ARRAY_INDEX";
private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
+ private static final String BATCH_LOOKUP = "BATCH_LOOKUP";
private final Properties properties = new Properties();
@@ -239,6 +240,14 @@
return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
}
+ public boolean isBatchLookupEnabled() {
+ return getBoolean(BATCH_LOOKUP, AlgebricksConfig.BATCH_LOOKUP_DEFAULT);
+ }
+
+ public void setBatchLookup(boolean batchedLookup) {
+ setBoolean(BATCH_LOOKUP, batchedLookup);
+ }
+
public void setExternalScanBufferSize(int bufferSize) {
setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
index e13ec30..129aba3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -132,6 +132,7 @@
e.setExecutionMode(inOp.getExecutionMode());
context.computeAndSetTypeEnvironmentForOperator(e);
e.recomputeSchema();
+ e.computeDeliveredPhysicalProperties(context);
inOpRef.setValue(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
index c34a671..3c829e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
@@ -28,6 +28,8 @@
public static final String INDEX_CURSOR_STATS = "INDEX_CURSOR_STATS";
+ public static final String TUPLE_PROJECTOR = "TUPLE_PROJECTOR";
+
private HyracksConstants() {
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
index 4732d71..1f5b441 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
@@ -43,7 +43,7 @@
//Based in part on LoggingHandler from Netty
public class CLFLogger extends ChannelDuplexHandler {
- private static final Logger accessLogger = LogManager.getLogger();
+ private static final Logger LOGGER = LogManager.getLogger();
private static final Level ACCESS_LOG_LEVEL = Level.forName("ACCESS", 550);
private static final DateTimeFormatter DATE_TIME_FORMATTER =
DateTimeFormatter.ofPattern("dd/MMM/yyyy:HH:mm:ss Z").withZone(ZoneId.systemDefault());
@@ -66,9 +66,14 @@
public void channelRead(ChannelHandlerContext ctx, Object msg) {
if (msg instanceof HttpRequest) {
HttpRequest req = (HttpRequest) msg;
- clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+ try {
+ clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+ } catch (Exception e) {
+ LOGGER.debug("ignoring {} obtaining client ip for {}", e, ctx.channel());
+ clientIp = "-";
+ }
requestTime = Instant.now();
- reqLine = req.method().toString() + " " + req.getUri() + " " + req.getProtocolVersion().toString();
+ reqLine = req.method().toString() + " " + req.uri() + " " + req.protocolVersion();
userAgentRef = headerValueOrDash("Referer", req) + " " + headerValueOrDash("User-Agent", req);
lastChunk = false;
}
@@ -116,7 +121,7 @@
}
private void printAndPrepare() {
- if (!accessLogger.isEnabled(ACCESS_LOG_LEVEL)) {
+ if (!LOGGER.isEnabled(ACCESS_LOG_LEVEL)) {
return;
}
logLineBuilder.append(clientIp);
@@ -131,7 +136,7 @@
logLineBuilder.append(" ").append(statusCode);
logLineBuilder.append(" ").append(respSize);
logLineBuilder.append(" ").append(userAgentRef);
- accessLogger.log(ACCESS_LOG_LEVEL, logLineBuilder);
+ LOGGER.log(ACCESS_LOG_LEVEL, logLineBuilder);
respSize = 0;
logLineBuilder.setLength(0);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
new file mode 100644
index 0000000..f436636
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+/**
+ * Allows stateful {@link DiskBTreePointSearchCursor} to resume the search without closing and reopening the cursor
+ * Note: upon closing a cursor, ensure the search state is cleared
+ */
+public interface IDiskBTreeStatefulPointSearchCursor {
+ int getLastPageId();
+
+ void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException;
+
+ ITreeIndexFrame getFrame();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 2455625..0ab88a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -29,10 +29,12 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
protected final int[] lowKeyFields;
protected final int[] highKeyFields;
@@ -52,6 +54,7 @@
protected byte[] searchCallbackProceedResultTrueValue;
protected final ITupleFilterFactory tupleFilterFactory;
protected final long outputLimit;
+ protected final ITupleProjectorFactory tupleProjectorFactory;
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -61,7 +64,8 @@
IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null,
+ DefaultTupleProjectorFactory.INSTANCE);
}
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -71,7 +75,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory tupleProjectorFactory) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -92,6 +96,7 @@
this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
+ this.tupleProjectorFactory = tupleProjectorFactory;
}
@Override
@@ -102,7 +107,7 @@
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
- searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, tupleProjectorFactory);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index fb331bf..a56e305 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
@@ -30,9 +31,11 @@
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
protected final boolean lowKeyInclusive;
@@ -52,7 +55,7 @@
this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
- false, null, null);
+ false, null, null, DefaultTupleProjectorFactory.INSTANCE);
}
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -62,11 +65,12 @@
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+ throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
- searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, projectorFactory);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
if (lowKeyFields != null && lowKeyFields.length > 0) {
@@ -111,7 +115,8 @@
@Override
protected void addAdditionalIndexAccessorParams(IIndexAccessParameters iap) throws HyracksDataException {
- // No additional parameters are required for the B+Tree search case
+ //Set tuple projector to get the information about the pushed down value accesses (if supported by the index)
+ iap.getParameters().put(HyracksConstants.TUPLE_PROJECTOR, tupleProjector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..40b2f5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -42,7 +42,6 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
import org.apache.hyracks.storage.am.common.api.IBTreeIndexTupleReference;
import org.apache.hyracks.storage.am.common.api.IPageManager;
-import org.apache.hyracks.storage.am.common.api.ISplitKey;
import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -51,7 +50,6 @@
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -64,17 +62,13 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.JSONUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
public class BTree extends AbstractTreeIndex {
public static final float DEFAULT_FILL_FACTOR = 0.7f;
@@ -85,7 +79,7 @@
private final AtomicInteger smoCounter;
private final ReadWriteLock treeLatch;
- private final int maxTupleSize;
+ protected final int maxTupleSize;
public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,
@@ -886,13 +880,13 @@
}
@Override
- public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new BTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
return createSearchCursor(exclusive);
}
@@ -1004,220 +998,7 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
- return new BTreeBulkLoader(fillFactor, verifyInput, callback);
- }
-
- public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
- protected final ISplitKey splitKey;
- protected final boolean verifyInput;
-
- public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
- throws HyracksDataException {
- super(fillFactor, callback);
- this.verifyInput = verifyInput;
- splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
- splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
- }
-
- @Override
- public void add(ITupleReference tuple) throws HyracksDataException {
- try {
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- NodeFrontier leafFrontier = nodeFrontiers.get(0);
- int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
- int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
- // try to free space by compression
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- leafFrame.compress();
- spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
- }
- //full, allocate new page
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- if (leafFrame.getTupleCount() == 0) {
- bufferCache.returnPage(leafFrontier.page, false);
- } else {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- if (verifyInput) {
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
- splitKey.setLeftPage(leafFrontier.pageId);
-
- propagateBulk(1, pagesToWrite);
-
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
-
- ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-
- write(leafFrontier.page);
- for (ICachedPage c : pagesToWrite) {
- write(c);
- }
- pagesToWrite.clear();
- splitKey.setRightPage(leafFrontier.pageId);
- }
- if (tupleSize > maxTupleSize) {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- // calculate required number of pages.
- int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
- final int multiplier =
- (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
- if (multiplier > 1) {
- leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
- freePageManager.takeBlock(metaFrame, multiplier - 1));
- } else {
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- }
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
- } else {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- }
- } else {
- if (verifyInput && leafFrame.getTupleCount() > 0) {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- }
- ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
- } catch (HyracksDataException | RuntimeException e) {
- logState(tuple, e);
- handleException();
- throw e;
- }
- }
-
- protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
- // New tuple should be strictly greater than last tuple.
- int cmpResult = cmp.compare(tuple, prevTuple);
- if (cmpResult < 0) {
- throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
- }
- if (cmpResult == 0) {
- throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
- }
- }
-
- protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
- if (splitKey.getBuffer() == null) {
- return;
- }
-
- if (level >= nodeFrontiers.size()) {
- addLevel();
- }
-
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
-
- ITupleReference tuple = splitKey.getTuple();
- int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
- int spaceNeeded = tupleBytes + slotSize + 4;
- if (tupleBytes > interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize())) {
- throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
- interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize()));
- }
-
- int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
- if (spaceUsed + spaceNeeded > interiorMaxBytes) {
-
- ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
- tuple = copyKey.getTuple();
-
- frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
- int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-
- ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- pagesToWrite.add(frontier.page);
- splitKey.setLeftPage(finalPageId);
-
- propagateBulk(level + 1, pagesToWrite);
- frontier.page = bufferCache.confiscatePage(BufferCache.INVALID_DPID);
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) level);
- }
- ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
- }
-
- private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
- if (level >= nodeFrontiers.size()) {
- rootPage = nodeFrontiers.get(level - 1).pageId;
- releasedLatches = true;
- return;
- }
- if (level < 1) {
- ICachedPage lastLeaf = nodeFrontiers.get(level).page;
- int lastLeafPage = nodeFrontiers.get(level).pageId;
- lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
- write(lastLeaf);
- nodeFrontiers.get(level).page = null;
- persistFrontiers(level + 1, lastLeafPage);
- return;
- }
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
- //just finalize = the layer right above the leaves has correct righthand pointers already
- if (rightPage < 0) {
- throw new HyracksDataException(
- "Error in index creation. Internal node appears to have no rightmost guide");
- }
- ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- write(frontier.page);
- frontier.pageId = finalPageId;
- persistFrontiers(level + 1, finalPageId);
- }
-
- @Override
- public void end() throws HyracksDataException {
- try {
- persistFrontiers(0, -1);
- super.end();
- } catch (HyracksDataException | RuntimeException e) {
- handleException();
- throw e;
- }
- }
-
- @Override
- public void abort() throws HyracksDataException {
- super.handleException();
- }
-
- private void logState(ITupleReference tuple, Exception e) {
- try {
- ObjectNode state = JSONUtil.createObject();
- state.set("leafFrame", leafFrame.getState());
- state.set("interiorFrame", interiorFrame.getState());
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- state.put("tupleSize", tupleSize);
- state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
- state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
- state.put("leafMaxBytes", leafMaxBytes);
- state.put("maxTupleSize", maxTupleSize);
- LOGGER.error("failed to add tuple {}", state, e);
- } catch (Throwable t) {
- e.addSuppressed(t);
- }
- }
+ return new BTreeNSMBulkLoader(fillFactor, verifyInput, callback, this);
}
@SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
new file mode 100644
index 0000000..04c84e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class BTreeNSMBulkLoader extends AbstractTreeIndexBulkLoader {
+ private static final Logger LOGGER = LogManager.getLogger();
+ protected final ISplitKey splitKey;
+ protected final boolean verifyInput;
+ private final int maxTupleSize;
+
+ public BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, verifyInput, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ super(fillFactor, callback, index, leafFrame);
+ this.verifyInput = verifyInput;
+ splitKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+ splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+ maxTupleSize = ((BTree) index).maxTupleSize;
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ try {
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+ int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+ // try to free space by compression
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ leafFrame.compress();
+ spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+ }
+ //full, allocate new page
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ if (leafFrame.getTupleCount() == 0) {
+ //The current page is empty. Return it.
+ bufferCache.returnPage(leafFrontier.page, false);
+ } else {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ if (verifyInput) {
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ //The current page is not empty. Write it.
+ writeFullLeafPage();
+ }
+ if (tupleSize > maxTupleSize) {
+ //We need a large page
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ // calculate required number of pages.
+ int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
+ final int multiplier =
+ (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
+ if (multiplier > 1) {
+ leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
+ freePageManager.takeBlock(metaFrame, multiplier - 1));
+ } else {
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ }
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
+ } else {
+ //allocate a new page
+ confiscateNewLeafPage();
+ }
+ } else {
+ if (verifyInput && leafFrame.getTupleCount() > 0) {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ }
+ ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+ } catch (HyracksDataException | RuntimeException e) {
+ logState(tuple, e);
+ handleException();
+ throw e;
+ }
+ }
+
+ protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
+ // New tuple should be strictly greater than last tuple.
+ int cmpResult = cmp.compare(tuple, prevTuple);
+ if (cmpResult < 0) {
+ throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
+ }
+ if (cmpResult == 0) {
+ throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
+ }
+ }
+
+ protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
+ if (splitKey.getBuffer() == null) {
+ return;
+ }
+
+ if (level >= nodeFrontiers.size()) {
+ addLevel();
+ }
+
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+
+ ITupleReference tuple = splitKey.getTuple();
+ int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+ int spaceNeeded = tupleBytes + slotSize + 4;
+ if (tupleBytes > interiorFrame.getMaxTupleSize(bufferCache.getPageSize())) {
+ throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
+ interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
+ }
+
+ int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+ if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+ ISplitKey copyKey = splitKey.duplicate(tupleWriter.createTupleReference());
+ tuple = copyKey.getTuple();
+
+ frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+ int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+
+ ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ pagesToWrite.add(frontier.page);
+ splitKey.setLeftPage(finalPageId);
+
+ propagateBulk(level + 1, pagesToWrite);
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) level);
+ }
+ ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+ }
+
+ private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
+ if (level >= nodeFrontiers.size()) {
+ setRootPageId(nodeFrontiers.get(level - 1).pageId);
+ releasedLatches = true;
+ return;
+ }
+ if (level < 1) {
+ ICachedPage lastLeaf = nodeFrontiers.get(level).page;
+ int lastLeafPage = nodeFrontiers.get(level).pageId;
+ lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, nodeFrontiers.get(level).pageId));
+ writeLastLeaf(lastLeaf);
+ nodeFrontiers.get(level).page = null;
+ persistFrontiers(level + 1, lastLeafPage);
+ return;
+ }
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+ //just finalize = the layer right above the leaves has correct righthand pointers already
+ if (rightPage < 0) {
+ throw new HyracksDataException("Error in index creation. Internal node appears to have no rightmost guide");
+ }
+ ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ write(frontier.page);
+ frontier.pageId = finalPageId;
+ persistFrontiers(level + 1, finalPageId);
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ try {
+ persistFrontiers(0, -1);
+ super.end();
+ } catch (HyracksDataException | RuntimeException e) {
+ handleException();
+ throw e;
+ }
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ super.handleException();
+ }
+
+ protected void writeFullLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ final int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+ splitKey.setLeftPage(leafFrontier.pageId);
+
+ propagateBulk(1, pagesToWrite);
+
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+
+ ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+
+ write(leafFrontier.page);
+ for (ICachedPage c : pagesToWrite) {
+ write(c);
+ }
+ pagesToWrite.clear();
+ splitKey.setRightPage(leafFrontier.pageId);
+ }
+
+ protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+ write(page);
+ }
+
+ protected final void confiscateNewLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ }
+
+ private void logState(ITupleReference tuple, Exception e) {
+ try {
+ ObjectNode state = JSONUtil.createObject();
+ state.set("leafFrame", leafFrame.getState());
+ state.set("interiorFrame", interiorFrame.getState());
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ state.put("tupleSize", tupleSize);
+ state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+ state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+ state.put("leafMaxBytes", leafMaxBytes);
+ state.put("maxTupleSize", maxTupleSize);
+ LOGGER.error("failed to add tuple {}", state, e);
+ } catch (Throwable t) {
+ e.addSuppressed(t);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..179f1da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -24,8 +24,8 @@
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.util.HyracksConstants;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -38,7 +38,6 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -76,7 +75,8 @@
private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
throws HyracksDataException {
ctx.reset();
- ctx.setPred((RangePredicate) searchPred);
+ RangePredicate rangePredicate = (RangePredicate) searchPred;
+ ctx.setPred(rangePredicate);
ctx.setCursor(cursor);
if (ctx.getPred().getLowKeyComparator() == null) {
ctx.getPred().setLowKeyComparator(ctx.getCmp());
@@ -87,25 +87,17 @@
cursor.setBufferCache(bufferCache);
cursor.setFileId(getFileId());
- if (cursor instanceof DiskBTreePointSearchCursor) {
- DiskBTreePointSearchCursor pointCursor = (DiskBTreePointSearchCursor) cursor;
+ if (cursor instanceof IDiskBTreeStatefulPointSearchCursor) {
+ IDiskBTreeStatefulPointSearchCursor pointCursor = (IDiskBTreeStatefulPointSearchCursor) cursor;
int lastPageId = pointCursor.getLastPageId();
- if (lastPageId != BufferCache.INVALID_PAGEID) {
- // check whether the last leaf page contains this key
- ICachedPage lastPage =
- bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), lastPageId), false);
- ctx.getLeafFrame().setPage(lastPage);
- if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), ctx.getLeafFrame())) {
- // use this page
- ctx.getCursorInitialState().setPage(lastPage);
- ctx.getCursorInitialState().setPageId(lastPageId);
- pointCursor.open(ctx.getCursorInitialState(), searchPred);
+ if (lastPageId != IBufferCache.INVALID_PAGEID) {
+ if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), pointCursor.getFrame())) {
+ pointCursor.setCursorToNextKey(searchPred);
return;
} else {
- // release the last page and clear the states of this cursor
+ // release the last page, clear the states of this cursor, and close the cursor
// then retry the search from root to leaf
- bufferCache.unpin(lastPage);
- pointCursor.clearSearchState();
+ cursor.close();
}
}
}
@@ -113,7 +105,7 @@
searchDown(rootNode, rootPage, ctx, cursor);
}
- private boolean fitInPage(ITupleReference key, MultiComparator comparator, IBTreeFrame frame)
+ private boolean fitInPage(ITupleReference key, MultiComparator comparator, ITreeIndexFrame frame)
throws HyracksDataException {
// assume that search keys are sorted (non-decreasing)
ITupleReference rightmostTuple = frame.getRightmostTuple();
@@ -191,14 +183,14 @@
}
@Override
- public DiskBTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new DiskBTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
@Override
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new DiskBTreePointSearchCursor(leafFrame, exclusive, stateful);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 1bf3ecf..8fd9a96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -21,13 +21,16 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
import org.apache.hyracks.storage.common.ICursorInitialState;
import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
+public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor
+ implements IDiskBTreeStatefulPointSearchCursor {
/**
* A stateful cursor keeps the search state (last search page Id + index) across multiple searches
* until {@link #clearSearchState()} is called explicity
@@ -36,16 +39,14 @@
private boolean nextHasBeenCalled;
- private int lastPageId = BufferCache.INVALID_PAGEID;
+ private int lastPageId;
private int lastTupleIndex = 0;
public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes, boolean stateful) {
super(frame, exclusiveLatchNodes);
this.stateful = stateful;
- }
-
- public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
- this(frame, exclusiveLatchNodes, false);
+ lastPageId = IBufferCache.INVALID_PAGEID;
+ lastTupleIndex = 0;
}
@Override
@@ -59,6 +60,12 @@
}
@Override
+ public void doClose() throws HyracksDataException {
+ clearSearchState();
+ super.doClose();
+ }
+
+ @Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (page != null) {
@@ -71,30 +78,10 @@
page = initialState.getPage();
isPageDirty = false;
frame.setPage(page);
-
- pred = (RangePredicate) searchPred;
- lowKeyCmp = pred.getLowKeyComparator();
- lowKey = pred.getLowKey();
-
- reusablePredicate.setLowKeyComparator(originalKeyCmp);
-
- lowKeyFtm = FindTupleMode.EXACT;
- lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
-
- nextHasBeenCalled = false;
-
- // only get the low key position
- tupleIndex = getLowKeyIndex();
- if (stateful) {
- lastPageId = pageId;
- if (tupleIndex >= 0) {
- lastTupleIndex = tupleIndex;
- } else {
- lastTupleIndex = -tupleIndex - 1;
- }
- }
+ setCursorToNextKey(searchPred);
}
+ @Override
public int getLastPageId() {
return lastPageId;
}
@@ -108,9 +95,38 @@
}
}
- public void clearSearchState() {
- this.lastPageId = BufferCache.INVALID_PAGEID;
+ @Override
+ public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+ pred = (RangePredicate) searchPred;
+ lowKeyCmp = pred.getLowKeyComparator();
+ lowKey = pred.getLowKey();
+
+ reusablePredicate.setLowKeyComparator(originalKeyCmp);
+
+ lowKeyFtm = FindTupleMode.EXACT;
+ lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
+
+ nextHasBeenCalled = false;
+
+ // only get the lowKey position
+ tupleIndex = getLowKeyIndex();
+ if (stateful) {
+ lastPageId = pageId;
+ if (tupleIndex >= 0) {
+ lastTupleIndex = tupleIndex;
+ } else {
+ lastTupleIndex = -tupleIndex - 1;
+ }
+ }
+ }
+
+ private void clearSearchState() {
+ this.lastPageId = IBufferCache.INVALID_PAGEID;
this.lastTupleIndex = 0;
}
+ @Override
+ public ITreeIndexFrame getFrame() {
+ return frame;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index eab8c96..1fe91d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -44,6 +44,7 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilter;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.tuples.ReferenceFrameTupleReference;
@@ -54,6 +55,8 @@
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
import org.apache.hyracks.util.IThreadStatsCollector;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -107,6 +110,7 @@
protected final long outputLimit;
protected long outputCount = 0;
protected boolean finished;
+ protected final ITupleProjector tupleProjector;
// no filter and limit pushdown
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -116,7 +120,7 @@
IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
- nonFilterWriterFactory, null, -1, false, null, null);
+ nonFilterWriterFactory, null, -1, false, null, null, DefaultTupleProjectorFactory.INSTANCE);
}
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -125,7 +129,8 @@
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+ throws HyracksDataException {
this.ctx = ctx;
this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
this.retainInput = retainInput;
@@ -162,6 +167,8 @@
if (this.tupleFilterFactory != null && this.retainMissing) {
throw new IllegalStateException("RetainMissing with tuple filter is not supported");
}
+
+ tupleProjector = projectorFactory.createTupleProjector(ctx);
}
protected abstract ISearchPredicate createSearchPredicate();
@@ -349,10 +356,7 @@
protected void writeTupleToOutput(ITupleReference tuple) throws IOException {
try {
- for (int i = 0; i < tuple.getFieldCount(); i++) {
- dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
+ tupleProjector.project(tuple, dos, tb);
} catch (Exception e) {
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 81e528b..11368bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -19,9 +19,6 @@
package org.apache.hyracks.storage.am.common.impls;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,17 +27,9 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
-import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -229,118 +218,6 @@
return fieldCount;
}
- public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
- protected final MultiComparator cmp;
- protected final int slotSize;
- protected final int leafMaxBytes;
- protected final int interiorMaxBytes;
- protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
- protected final ITreeIndexMetadataFrame metaFrame;
- protected final ITreeIndexTupleWriter tupleWriter;
- protected ITreeIndexFrame leafFrame;
- protected ITreeIndexFrame interiorFrame;
- // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
- // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
- // that tree are allowed. Currently, this is not enforced.
- protected boolean releasedLatches;
- private final IFIFOPageWriter pageWriter;
- protected List<ICachedPage> pagesToWrite;
- private final ICompressedPageWriter compressedPageWriter;
-
- public AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- leafFrame = leafFrameFactory.createFrame();
- interiorFrame = interiorFrameFactory.createFrame();
- metaFrame = freePageManager.createMetadataFrame();
-
- pageWriter = bufferCache.createFIFOWriter(callback, this);
-
- if (!isEmptyTree(leafFrame)) {
- throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
- }
-
- this.cmp = MultiComparator.create(cmpFactories);
-
- leafFrame.setMultiComparator(cmp);
- interiorFrame.setMultiComparator(cmp);
-
- tupleWriter = leafFrame.getTupleWriter();
-
- NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
- leafFrontier.page =
- bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
-
- interiorFrame.setPage(leafFrontier.page);
- interiorFrame.initBuffer((byte) 0);
- interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
-
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
- slotSize = leafFrame.getSlotSize();
-
- nodeFrontiers.add(leafFrontier);
- pagesToWrite = new ArrayList<>();
- compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
- }
-
- protected void handleException() {
- // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
- compressedPageWriter.abort();
- for (NodeFrontier nodeFrontier : nodeFrontiers) {
- if (nodeFrontier != null && nodeFrontier.page != null) {
- ICachedPage frontierPage = nodeFrontier.page;
- if (frontierPage.confiscated()) {
- bufferCache.returnPage(frontierPage, false);
- }
- }
- }
- for (ICachedPage pageToDiscard : pagesToWrite) {
- if (pageToDiscard != null) {
- bufferCache.returnPage(pageToDiscard, false);
- }
- }
- releasedLatches = true;
- }
-
- @Override
- public void end() throws HyracksDataException {
- if (hasFailed()) {
- throw HyracksDataException.create(getFailure());
- }
- freePageManager.setRootPageId(rootPage);
- }
-
- protected void addLevel() throws HyracksDataException {
- NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
- frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
- frontier.pageId = -1;
- frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) nodeFrontiers.size());
- nodeFrontiers.add(frontier);
- }
-
- public ITreeIndexFrame getLeafFrame() {
- return leafFrame;
- }
-
- public void setLeafFrame(ITreeIndexFrame leafFrame) {
- this.leafFrame = leafFrame;
- }
-
- public void write(ICachedPage cPage) throws HyracksDataException {
- compressedPageWriter.prepareWrite(cPage);
- pageWriter.write(cPage);
- }
-
- @Override
- public void force() throws HyracksDataException {
- bufferCache.force(fileId, false);
- }
-
- }
-
public IBinaryComparatorFactory[] getCmpFactories() {
return cmpFactories;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
new file mode 100644
index 0000000..45a88a7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
+ protected final IBufferCache bufferCache;
+ protected final IPageManager freePageManager;
+ protected final AbstractTreeIndex treeIndex;
+ protected final int fileId;
+ protected final MultiComparator cmp;
+ protected final int slotSize;
+ protected final int leafMaxBytes;
+ protected final int interiorMaxBytes;
+ protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
+ protected final ITreeIndexMetadataFrame metaFrame;
+ protected final ITreeIndexTupleWriter tupleWriter;
+ protected ITreeIndexFrame leafFrame;
+ protected ITreeIndexFrame interiorFrame;
+ // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
+ // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
+ // that tree are allowed. Currently, this is not enforced.
+ protected boolean releasedLatches;
+ private final IFIFOPageWriter pageWriter;
+ protected List<ICachedPage> pagesToWrite;
+ private final ICompressedPageWriter compressedPageWriter;
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ this.bufferCache = index.getBufferCache();
+ this.freePageManager = index.getPageManager();
+ this.fileId = index.getFileId();
+ this.treeIndex = (AbstractTreeIndex) index;
+ this.leafFrame = leafFrame;
+ interiorFrame = treeIndex.getInteriorFrameFactory().createFrame();
+ metaFrame = freePageManager.createMetadataFrame();
+
+ pageWriter = bufferCache.createFIFOWriter(callback, this);
+
+ if (!treeIndex.isEmptyTree(leafFrame)) {
+ throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
+ }
+
+ this.cmp = MultiComparator.create(treeIndex.getCmpFactories());
+
+ leafFrame.setMultiComparator(cmp);
+ interiorFrame.setMultiComparator(cmp);
+
+ tupleWriter = leafFrame.getTupleWriter();
+ NodeFrontier leafFrontier = new NodeFrontier(createTupleReference());
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+ leafFrontier.page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
+
+ interiorFrame.setPage(leafFrontier.page);
+ interiorFrame.initBuffer((byte) 0);
+ interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
+
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
+ slotSize = leafFrame.getSlotSize();
+
+ nodeFrontiers.add(leafFrontier);
+ pagesToWrite = new ArrayList<>();
+ compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
+ }
+
+ protected ITreeIndexTupleReference createTupleReference() {
+ return leafFrame.createTupleReference();
+ }
+
+ protected void handleException() {
+ // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
+ compressedPageWriter.abort();
+ for (NodeFrontier nodeFrontier : nodeFrontiers) {
+ if (nodeFrontier != null && nodeFrontier.page != null) {
+ ICachedPage frontierPage = nodeFrontier.page;
+ if (frontierPage.confiscated()) {
+ bufferCache.returnPage(frontierPage, false);
+ }
+ }
+ }
+ for (ICachedPage pageToDiscard : pagesToWrite) {
+ if (pageToDiscard != null) {
+ bufferCache.returnPage(pageToDiscard, false);
+ }
+ }
+ releasedLatches = true;
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ if (hasFailed()) {
+ throw HyracksDataException.create(getFailure());
+ }
+ freePageManager.setRootPageId(treeIndex.getRootPageId());
+ }
+
+ protected void setRootPageId(int rootPage) {
+ treeIndex.rootPage = rootPage;
+ }
+
+ protected void addLevel() throws HyracksDataException {
+ NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ frontier.pageId = -1;
+ frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) nodeFrontiers.size());
+ nodeFrontiers.add(frontier);
+ }
+
+ public ITreeIndexFrame getLeafFrame() {
+ return leafFrame;
+ }
+
+ public void setLeafFrame(ITreeIndexFrame leafFrame) {
+ this.leafFrame = leafFrame;
+ }
+
+ public void write(ICachedPage cPage) throws HyracksDataException {
+ compressedPageWriter.prepareWrite(cPage);
+ pageWriter.write(cPage);
+ }
+
+ @Override
+ public void force() throws HyracksDataException {
+ bufferCache.force(fileId, false);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
new file mode 100644
index 0000000..00cb0c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+class DefaultTupleProjector implements ITupleProjector {
+ public static final ITupleProjector INSTANCE = new DefaultTupleProjector();
+
+ private DefaultTupleProjector() {
+ }
+
+ @Override
+ public void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
new file mode 100644
index 0000000..092982d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
+
+public class DefaultTupleProjectorFactory implements ITupleProjectorFactory {
+ private static final long serialVersionUID = -4525893018744087821L;
+ public static final DefaultTupleProjectorFactory INSTANCE = new DefaultTupleProjectorFactory();
+
+ private DefaultTupleProjectorFactory() {
+ }
+
+ @Override
+ public ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException {
+ return DefaultTupleProjector.INSTANCE;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 3800d17..3e03e5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class LSMBTreeBatchPointSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
private static final long serialVersionUID = 1L;
@@ -38,10 +39,11 @@
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory,
- long outputLimit) {
+ long outputLimit, ITupleProjectorFactory tupleProjectorFactory) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
+ maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null,
+ tupleProjectorFactory);
}
@Override
@@ -51,7 +53,7 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, tupleFilterFactory,
- outputLimit);
+ outputLimit, tupleProjectorFactory);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index 30813ef..9b8c353 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -34,10 +34,12 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
public class LSMBTreeBatchPointSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
@@ -48,18 +50,19 @@
boolean highKeyInclusive, int[] minFilterKeyFields, int[] maxFilterKeyFields,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, ITupleProjectorFactory tupleProjectorFactory)
+ throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
- null);
+ null, tupleProjectorFactory);
this.keyFields = lowKeyFields;
}
@Override
protected IIndexCursor createCursor() throws HyracksDataException {
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
- return new LSMBTreeBatchPointSearchCursor(lsmAccessor.getOpContext());
+ return ((LSMBTree) index).createBatchPointSearchCursor(lsmAccessor.getOpContext());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index c3d1416..d2fbbef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -341,7 +341,7 @@
try {
List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
long numElements = getNumberOfElements(mergedComponents);
- mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
+ mergedComponent = createDiskComponent(getMergeComponentFactory(), mergeOp.getTarget(), null,
mergeOp.getBloomFilterTarget(), true);
IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
@@ -418,7 +418,7 @@
}
public ILSMIndexAccessor createAccessor(AbstractLSMIndexOperationContext opCtx) {
- return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
+ return new LSMTreeIndexAccessor(getHarness(), opCtx, getCursorFactory());
}
@Override
@@ -483,8 +483,28 @@
returnDeletedTuples = true;
}
IIndexCursorStats stats = new IndexCursorStats();
- LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ LSMBTreeRangeSearchCursor cursor = createCursor(opCtx, returnDeletedTuples, stats);
return new LSMBTreeMergeOperation(accessor, cursor, stats, mergeFileRefs.getInsertIndexFileReference(),
mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
}
+
+ public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+ return new LSMBTreeBatchPointSearchCursor(opCtx);
+ }
+
+ protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+ boolean returnDeletedTuples, IIndexCursorStats stats) {
+ return new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ }
+
+ /**
+ * @return Merge component factory (could be different from {@link #componentFactory}
+ */
+ protected ILSMDiskComponentFactory getMergeComponentFactory() {
+ return componentFactory;
+ }
+
+ protected ICursorFactory getCursorFactory() {
+ return cursorFactory;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
index 8ab6fb1..73d06d2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
@@ -21,14 +21,15 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import org.apache.hyracks.storage.am.btree.impls.BatchPredicate;
-import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
/**
* This cursor performs point searches for each batch of search keys.
* Assumption: the search keys must be sorted into the increasing order.
- *
*/
public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
@@ -42,7 +43,9 @@
while (!foundTuple && batchPred.hasNext()) {
batchPred.next();
if (foundIn >= 0) {
- btreeCursors[foundIn].close();
+ if (operationalComponents.get(foundIn).getType() == LSMComponentType.MEMORY) {
+ btreeCursors[foundIn].close();
+ }
foundIn = -1;
}
foundTuple = super.doHasNext();
@@ -56,6 +59,11 @@
}
@Override
+ protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+ return btreeAccessor.createPointCursor(false, true);
+ }
+
+ @Override
protected boolean isSearchCandidate(int componentIndex) throws HyracksDataException {
if (!super.isSearchCandidate(componentIndex)) {
return false;
@@ -68,21 +76,6 @@
maxFileterKey, opCtx.getFilterCmp());
}
- @Override
- protected void closeCursors() throws HyracksDataException {
- super.closeCursors();
- if (btreeCursors != null) {
- // clear search states of btree cursors
- for (int i = 0; i < numBTrees; ++i) {
- if (btreeCursors[i] != null) {
- if (btreeCursors[i] instanceof DiskBTreePointSearchCursor) {
- ((DiskBTreePointSearchCursor) btreeCursors[i]).clearSearchState();
- }
- }
- }
- }
- }
-
public int getKeyIndex() {
return ((BatchPredicate) predicate).getKeyIndex();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1312e30..a00e10e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -43,7 +43,7 @@
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.util.trace.ITracer;
-public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
+public class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
/*
* Finals
@@ -74,9 +74,9 @@
IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer) {
super(index, btreeFields, filterFields, filterCmpFactories, searchCallback, modificationCallback, tracer);
LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
- IBinaryComparatorFactory cmpFactories[] = c.getIndex().getComparatorFactories();
+ IBinaryComparatorFactory[] cmpFactories = c.getIndex().getComparatorFactories();
if (cmpFactories[0] != null) {
- this.cmp = MultiComparator.create(c.getIndex().getComparatorFactories());
+ this.cmp = createMultiComparator(c.getIndex().getComparatorFactories());
} else {
this.cmp = null;
}
@@ -112,6 +112,10 @@
insertSearchCursor = new LSMBTreePointSearchCursor(this);
}
+ protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+ return MultiComparator.create(cmpFactories);
+ }
+
@Override
public void setOperation(IndexOperation newOp) {
reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index d4903d9..9740bc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -199,7 +199,8 @@
for (int i = 0; i < numBTrees; i++) {
ILSMComponent component = operationalComponents.get(i);
BTree btree = (BTree) component.getIndex();
- if (component.getType() == LSMComponentType.MEMORY) {
+ LSMComponentType type = component.getType();
+ if (type == LSMComponentType.MEMORY) {
includeMutableComponent = true;
if (bloomFilters[i] != null) {
destroyAndNullifyCursorAtIndex(i);
@@ -212,8 +213,8 @@
}
if (btreeAccessors[i] == null) {
- btreeAccessors[i] = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- btreeCursors[i] = btreeAccessors[i].createPointCursor(false, false);
+ btreeAccessors[i] = createAccessor(type, btree, i);
+ btreeCursors[i] = createCursor(type, btreeAccessors[i]);
} else {
// re-use
btreeAccessors[i].reset(btree, NoOpIndexAccessParameters.INSTANCE);
@@ -225,6 +226,14 @@
hashComputed = false;
}
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int i) throws HyracksDataException {
+ return btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ }
+
+ protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+ return btreeAccessor.createPointCursor(false, false);
+ }
+
private void destroyAndNullifyCursorAtIndex(int i) throws HyracksDataException {
// component at location i was a disk component before, and is now a memory component, or vise versa
bloomFilters[i] = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 2c5fb50..968416c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -49,7 +49,7 @@
private final RangePredicate reusablePred;
private ISearchOperationCallback searchCallback;
private BTreeAccessor[] btreeAccessors;
- private boolean[] isMemoryComponent;
+ protected boolean[] isMemoryComponent;
private ArrayTupleBuilder tupleBuilder;
private boolean canCallProceed = true;
private boolean resultOfSearchCallbackProceed = false;
@@ -149,6 +149,7 @@
// There are no more elements in the memory component.. can safely skip locking for the
// remaining operations
includeMutableComponent = false;
+ excludeMemoryComponent();
}
}
} else {
@@ -180,6 +181,7 @@
// the tree of head tuple
// the head element of PQ is useless now
PriorityQueueElement e = outputPriorityQueue.poll();
+ markAsDeleted(e);
pushIntoQueueFromCursorAndReplaceThisElement(e);
} else {
// If the previous tuple and the head tuple are different
@@ -200,6 +202,14 @@
}
+ protected void excludeMemoryComponent() {
+ //NoOp
+ }
+
+ protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+ //NoOp
+ }
+
private void pushOutputElementIntoQueueIfNeeded() throws HyracksDataException {
if (needPushElementIntoQueue) {
pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
@@ -250,7 +260,7 @@
}
}
- private int replaceFrom() throws HyracksDataException {
+ protected int replaceFrom() throws HyracksDataException {
int replaceFrom = -1;
if (!switchPossible) {
return replaceFrom;
@@ -386,20 +396,21 @@
}
for (int i = 0; i < numBTrees; i++) {
ILSMComponent component = operationalComponents.get(i);
+ LSMComponentType type = component.getType();
BTree btree;
if (component.getType() == LSMComponentType.MEMORY) {
includeMutableComponent = true;
}
btree = (BTree) component.getIndex();
if (btreeAccessors[i] == null || destroyIncompatible(component, i)) {
- btreeAccessors[i] = btree.createAccessor(iap);
- rangeCursors[i] = btreeAccessors[i].createSearchCursor(false);
+ btreeAccessors[i] = createAccessor(type, btree, i);
+ rangeCursors[i] = createCursor(type, btreeAccessors[i]);
} else {
// re-use
btreeAccessors[i].reset(btree, iap);
rangeCursors[i].close();
}
- isMemoryComponent[i] = component.getType() == LSMComponentType.MEMORY;
+ isMemoryComponent[i] = type == LSMComponentType.MEMORY;
}
IndexCursorUtils.open(btreeAccessors, rangeCursors, searchPred);
try {
@@ -433,4 +444,12 @@
return resultOfSearchCallbackProceed;
}
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+ return btree.createAccessor(iap);
+ }
+
+ protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+ return accessor.createSearchCursor(false);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index efacad1..aa72267 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -46,6 +46,13 @@
scanCursor = new LSMBTreeDiskComponentScanCursor(opCtx);
}
+ protected LSMBTreeSearchCursor(LSMBTreePointSearchCursor pointCursor, LSMBTreeRangeSearchCursor rangeCursor,
+ LSMBTreeDiskComponentScanCursor scanCursor) {
+ this.pointCursor = pointCursor;
+ this.rangeCursor = rangeCursor;
+ this.scanCursor = scanCursor;
+ }
+
@Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 13a0e27..acb84e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -20,7 +20,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index b7eb115..11385de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -50,7 +51,7 @@
protected boolean open;
protected RTreeSearchCursor[] rtreeCursors;
- protected BTreeRangeSearchCursor[] btreeCursors;
+ protected ITreeIndexCursor[] btreeCursors;
protected RTreeAccessor[] rtreeAccessors;
protected BTreeAccessor[] btreeAccessors;
protected BloomFilter[] bloomFilters;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 8e5cb35..729ca74 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -142,7 +142,8 @@
bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(), linearizerArray,
btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
- BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
+ BTreeRangeSearchCursor btreeScanCursor =
+ (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false);
try {
isEmpty = true;
memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index a8a4252..e1c6f5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
import org.apache.hyracks.storage.am.rtree.util.RTreeUtils;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -59,7 +60,7 @@
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
nonFilterWriterFactory, null, -1, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
- searchCallbackProceedResultTrueValue);
+ searchCallbackProceedResultTrueValue, DefaultTupleProjectorFactory.INSTANCE);
if (keyFields != null && keyFields.length > 0) {
searchKey = new PermutingFrameTupleReference();
searchKey.setFieldPermutation(keyFields);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 7e8f249..d85200f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -41,6 +41,7 @@
import org.apache.hyracks.storage.am.common.frames.AbstractSlotManager;
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -902,7 +903,7 @@
return new RTreeBulkLoader(fillFactor, callback);
}
- public class RTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
+ public class RTreeBulkLoader extends AbstractTreeIndexBulkLoader {
ITreeIndexFrame lowerFrame, prevInteriorFrame;
RTreeTypeAwareTupleWriter interiorFrameTupleWriter =
((RTreeTypeAwareTupleWriter) interiorFrame.getTupleWriter());
@@ -911,7 +912,7 @@
List<Integer> prevNodeFrontierPages = new ArrayList<>();
public RTreeBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- super(fillFactor, callback);
+ super(fillFactor, callback, RTree.this);
prevInteriorFrame = interiorFrameFactory.createFrame();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
new file mode 100644
index 0000000..8ca1a82
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.projection;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ITupleProjector {
+ void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
new file mode 100644
index 0000000..ff9ecf9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.projection;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Tuple projector allows the data source to project the values needed before it pushed up
+ * to the upper operator.
+ */
+public interface ITupleProjectorFactory extends Serializable {
+ ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException;
+}