You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/11/05 05:12:41 UTC

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Ali Alsuliman has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/3020

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................

[ASTERIXDB-2474][COMP] Remove physical optimization static variable

- user model changes: no
- storage format changes: no
- interface changes: no

details:
This patch is to remove the physical optimization config static
variable used in IndexUtil to prevent side effects.

Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
13 files changed, 180 insertions(+), 170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/20/3020/1

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 7b5eb83..98feafa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.algebra.operators.physical;
 
+import java.util.Map;
+
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSourceId;
@@ -111,7 +114,9 @@
             retainNull = true;
         }
         // In-memory budget (frame limit) for inverted-index search operations
-        int frameLimit = OptimizationConfUtil.getPhysicalOptimizationConfig().getMaxFramesForTextSearch();
+        CompilerProperties compilerProp = metadataProvider.getApplicationContext().getCompilerProperties();
+        Map<String, Object> queryConfig = metadataProvider.getConfig();
+        int frameLimit = OptimizationConfUtil.getTextSearchNumFrames(compilerProp, queryConfig, op.getSourceLocation());
 
         // Build runtime.
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch =
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 f9ad13f..8344d41 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
@@ -36,6 +36,7 @@
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.api.INodeJobTracker;
 import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -123,11 +124,6 @@
  */
 public class APIFramework {
 
-    private static final int MIN_FRAME_LIMIT_FOR_SORT = 3;
-    private static final int MIN_FRAME_LIMIT_FOR_GROUP_BY = 4;
-    private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
-    // one for query, two for intermediate results, one for final result, and one for reading an inverted list
-    private static final int MIN_FRAME_LIMIT_FOR_TEXTSEARCH = 5;
     private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
 
     // A white list of supported configurable parameters.
@@ -225,7 +221,7 @@
         CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
         Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
         final PhysicalOptimizationConfig physOptConf =
-                getPhysicalOptimizationConfig(compilerProperties, querySpecificConfig, sourceLoc);
+                OptimizationConfUtil.createPhysicalOptimizationConf(compilerProperties, querySpecificConfig, sourceLoc);
 
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
@@ -319,36 +315,6 @@
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
-    }
-
-    protected PhysicalOptimizationConfig getPhysicalOptimizationConfig(CompilerProperties compilerProperties,
-            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
-        int frameSize = compilerProperties.getFrameSize();
-        int sortFrameLimit = getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
-                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT, sourceLoc);
-        int groupFrameLimit = getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
-                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
-        int joinFrameLimit = getFrameLimit(CompilerProperties.COMPILER_JOINMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
-                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN, sourceLoc);
-        int textSearchFrameLimit = getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
-                compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH, sourceLoc);
-        int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig);
-        boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
-
-        final PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
-        physOptConf.setFrameSize(frameSize);
-        physOptConf.setMaxFramesExternalSort(sortFrameLimit);
-        physOptConf.setMaxFramesExternalGroupBy(groupFrameLimit);
-        physOptConf.setMaxFramesForJoin(joinFrameLimit);
-        physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
-        physOptConf.setSortParallel(fullParallelSort);
-        physOptConf.setSortSamples(sortNumSamples);
-
-        return physOptConf;
     }
 
     protected IPrinterFactoryProvider getPrinterFactoryProvider(IDataFormat format,
@@ -475,43 +441,10 @@
         return ncMap.values().stream().mapToInt(NodeControllerInfo::getNumAvailableCores).sum();
     }
 
-    // Gets the frame limit.
-    private static int getFrameLimit(String parameterName, String parameter, long memBudgetInConfiguration,
-            int frameSize, int minFrameLimit, SourceLocation sourceLoc) throws AlgebricksException {
-        IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
-        long memBudget;
-        try {
-            memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
-        } catch (IllegalArgumentException e) {
-            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
-        }
-        int frameLimit = (int) (memBudget / frameSize);
-        if (frameLimit < minFrameLimit) {
-            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc, parameterName,
-                    frameSize * minFrameLimit);
-        }
-        // Sets the frame limit to the minimum frame limit if the caculated frame limit is too small.
-        return Math.max(frameLimit, minFrameLimit);
-    }
-
     // Gets the parallelism parameter.
     private static int getParallelism(String parameter, int parallelismInConfiguration) {
         IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.UNSIGNED_INTEGER;
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
-    }
-
-    private boolean getSortParallel(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
-        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_PARALLEL_KEY);
-        if (valueInQuery != null) {
-            return OptionTypes.BOOLEAN.parse(valueInQuery);
-        }
-        return compilerProperties.getSortParallel();
-    }
-
-    private int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
-        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
-        return valueInQuery == null ? compilerProperties.getSortSamples()
-                : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
     }
 
     // Validates if the query contains unsupported query parameters.
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 0471b25..4484b00 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
@@ -18,13 +18,99 @@
  */
 package org.apache.asterix.common.config;
 
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.control.common.config.OptionTypes;
 
 public class OptimizationConfUtil {
 
-    private static final PhysicalOptimizationConfig physicalOptimizationConfig = new PhysicalOptimizationConfig();
+    private static final int MIN_FRAME_LIMIT_FOR_SORT = 3;
+    private static final int MIN_FRAME_LIMIT_FOR_GROUP_BY = 4;
+    private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
+    // one for query, two for intermediate results, one for final result, and one for reading an inverted list
+    private static final int MIN_FRAME_LIMIT_FOR_TEXT_SEARCH = 5;
 
-    public static PhysicalOptimizationConfig getPhysicalOptimizationConfig() {
-        return physicalOptimizationConfig;
+    private OptimizationConfUtil() {
+    }
+
+    public static PhysicalOptimizationConfig createPhysicalOptimizationConf(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+        int frameSize = compilerProperties.getFrameSize();
+        int sortFrameLimit = getSortNumFrames(compilerProperties, querySpecificConfig, sourceLoc);
+        int groupFrameLimit = getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
+                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
+        int joinFrameLimit = getFrameLimit(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
+                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN, sourceLoc);
+        int textSearchFrameLimit = getTextSearchNumFrames(compilerProperties, querySpecificConfig, sourceLoc);
+        int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig);
+        boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
+
+        PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
+        physOptConf.setFrameSize(frameSize);
+        physOptConf.setMaxFramesExternalSort(sortFrameLimit);
+        physOptConf.setMaxFramesExternalGroupBy(groupFrameLimit);
+        physOptConf.setMaxFramesForJoin(joinFrameLimit);
+        physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
+        physOptConf.setSortParallel(fullParallelSort);
+        physOptConf.setSortSamples(sortNumSamples);
+
+        return physOptConf;
+    }
+
+    public static int getSortNumFrames(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        return getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
+                compilerProperties.getSortMemorySize(), compilerProperties.getFrameSize(), MIN_FRAME_LIMIT_FOR_SORT,
+                sourceLoc);
+    }
+
+    public static int getTextSearchNumFrames(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+        return getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
+                compilerProperties.getTextSearchMemorySize(), compilerProperties.getFrameSize(),
+                MIN_FRAME_LIMIT_FOR_TEXT_SEARCH, sourceLoc);
+    }
+
+    private static int getFrameLimit(String parameterName, String parameter, long memBudgetInConfiguration,
+            int frameSize, int minFrameLimit, SourceLocation sourceLoc) throws AlgebricksException {
+        IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
+        long memBudget;
+        try {
+            memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
+        }
+        int frameLimit = (int) (memBudget / frameSize);
+        if (frameLimit < minFrameLimit) {
+            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc, parameterName,
+                    frameSize * minFrameLimit);
+        }
+        // sets the frame limit to the minimum frame limit if the calculated frame limit is too small.
+        return Math.max(frameLimit, minFrameLimit);
+    }
+
+    private static boolean getSortParallel(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_PARALLEL_KEY);
+        if (valueInQuery != null) {
+            return OptionTypes.BOOLEAN.parse(valueInQuery);
+        }
+        return compilerProperties.getSortParallel();
+    }
+
+    private static int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
+        return valueInQuery == null ? compilerProperties.getSortSamples()
+                : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index c29279b..311e4b6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -25,7 +25,6 @@
 import java.util.Set;
 
 import org.apache.asterix.common.config.DatasetConfig;
-import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.TxnId;
@@ -36,7 +35,6 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
@@ -46,8 +44,6 @@
 
     //TODO: replace this null with an empty array. currently, this breaks many tests
     private static final int[] empty = null;
-    private static final PhysicalOptimizationConfig physicalOptimizationConfig =
-            OptimizationConfUtil.getPhysicalOptimizationConfig();
 
     private IndexUtil() {
     }
@@ -105,22 +101,22 @@
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(options);
     }
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCreationJobSpec();
     }
 
@@ -135,10 +131,10 @@
         SecondaryIndexOperationsHelper secondaryIndexHelper;
         if (dataset.isCorrelated()) {
             secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
-                    index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+                    index, metadataProvider, sourceLoc);
         } else {
             secondaryIndexHelper = SecondaryTreeIndexOperationsHelper.createIndexOperationsHelper(dataset, index,
-                    metadataProvider, physicalOptimizationConfig, sourceLoc);
+                    metadataProvider, sourceLoc);
         }
         if (files != null) {
             secondaryIndexHelper.setExternalFiles(files);
@@ -148,8 +144,8 @@
 
     public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCompactJobSpec();
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 56e967e..36918d2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -33,7 +33,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
@@ -46,6 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -56,9 +56,9 @@
 
 public class SecondaryBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
 
-    protected SecondaryBTreeOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -292,7 +292,7 @@
 
     }
 
-    protected int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
+    private int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
         int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
         for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 12e21f6..3930563 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -31,7 +31,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
@@ -44,6 +43,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -56,9 +56,9 @@
  */
 public class SecondaryCorrelatedBTreeOperationsHelper extends SecondaryCorrelatedTreeIndexOperationsHelper {
 
-    protected SecondaryCorrelatedBTreeOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryCorrelatedBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index c6e4322..4699c82 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -33,7 +33,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -45,6 +44,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ShortPointable;
@@ -73,8 +73,8 @@
     private int[] secondaryFilterFieldsForNonBulkLoadOps;
 
     protected SecondaryCorrelatedInvertedIndexOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -305,9 +305,8 @@
         for (int i = 0; i < numPrimaryKeys; i++) {
             taggedSortFields[idx++] = i + numSecondaryKeys + NUM_TAG_FIELDS;
         }
-        ExternalSortOperatorDescriptor sortOp =
-                new ExternalSortOperatorDescriptor(spec, physOptConf.getMaxFramesExternalSort(), taggedSortFields,
-                        taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames,
+                taggedSortFields, taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index 7a855d5..c870c60 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -37,7 +37,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
@@ -46,6 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -61,9 +61,9 @@
     protected boolean isPointMBR;
     protected RecordDescriptor secondaryRecDescForPointMBR = null;
 
-    protected SecondaryCorrelatedRTreeOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryCorrelatedRTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index c8d0bf3..b044293 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -34,7 +34,6 @@
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
@@ -114,8 +113,8 @@
     }
 
     protected SecondaryCorrelatedTreeIndexOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     protected RecordDescriptor getTaggedRecordDescriptor(RecordDescriptor recDescriptor) {
@@ -267,9 +266,8 @@
         for (int i = 1; i < taggedSortFields.length; i++) {
             taggedSortFields[i] = i + 1;
         }
-        ExternalSortOperatorDescriptor sortOp =
-                new ExternalSortOperatorDescriptor(spec, physOptConf.getMaxFramesExternalSort(), taggedSortFields,
-                        taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames,
+                taggedSortFields, taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
         sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
@@ -295,7 +293,7 @@
     }
 
     protected IOperatorDescriptor createPrimaryIndexScanDiskComponentsOp(JobSpecification spec,
-            MetadataProvider metadataProvider, RecordDescriptor outRecDesc) throws AlgebricksException {
+            MetadataProvider metadataProvider, RecordDescriptor outRecDesc) {
         ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
         ISearchOperationCallbackFactory searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(
                 dataset.getDatasetId(), dataset.getPrimaryBloomFilterFields(), txnSubsystemProvider,
@@ -311,31 +309,29 @@
     }
 
     public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
-            throws AlgebricksException {
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case BTREE:
                 indexOperationsHelper =
-                        new SecondaryCorrelatedBTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                        new SecondaryCorrelatedBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case RTREE:
                 indexOperationsHelper =
-                        new SecondaryCorrelatedRTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                        new SecondaryCorrelatedRTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
                 indexOperationsHelper = new SecondaryCorrelatedInvertedIndexOperationsHelper(dataset, index,
-                        physOptConf, metadataProvider);
+                        metadataProvider, sourceLoc);
                 break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
                         index.getIndexType());
         }
-        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 5f9e6ef..7df723b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -26,6 +26,7 @@
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -55,7 +56,6 @@
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -82,7 +82,6 @@
 // possible index job specs,
 // not just for creation.
 public abstract class SecondaryIndexOperationsHelper {
-    protected final PhysicalOptimizationConfig physOptConf;
     protected final MetadataProvider metadataProvider;
     protected final Dataset dataset;
     protected final Index index;
@@ -119,14 +118,14 @@
     protected int[] secondaryBTreeFields;
     protected List<ExternalFile> externalFiles;
     protected int numPrimaryKeys;
-    protected SourceLocation sourceLoc;
+    protected final SourceLocation sourceLoc;
+    protected final int sortNumFrames;
 
     // Prevent public construction. Should be created via createIndexCreator().
-    protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
+    protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
         this.dataset = dataset;
         this.index = index;
-        this.physOptConf = physOptConf;
         this.metadataProvider = metadataProvider;
         this.itemType =
                 (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
@@ -134,6 +133,8 @@
         Pair<ARecordType, ARecordType> enforcedTypes = getEnforcedType(index, itemType, metaType);
         this.enforcedItemType = enforcedTypes.first;
         this.enforcedMetaType = enforcedTypes.second;
+        this.sourceLoc = sourceLoc;
+        this.sortNumFrames = getSortNumFrames(metadataProvider, sourceLoc);
     }
 
     private static Pair<ARecordType, ARecordType> getEnforcedType(Index index, ARecordType aRecordType,
@@ -143,32 +144,34 @@
                 : new Pair<>(null, null);
     }
 
-    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
+    private static int getSortNumFrames(MetadataProvider metadataProvider, SourceLocation sourceLoc)
             throws AlgebricksException {
+        return OptimizationConfUtil.getSortNumFrames(metadataProvider.getApplicationContext().getCompilerProperties(),
+                metadataProvider.getConfig(), sourceLoc);
+    }
+
+    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case BTREE:
-                indexOperationsHelper =
-                        new SecondaryBTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                indexOperationsHelper = new SecondaryBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case RTREE:
-                indexOperationsHelper =
-                        new SecondaryRTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                indexOperationsHelper = new SecondaryRTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
                 indexOperationsHelper =
-                        new SecondaryInvertedIndexOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                        new SecondaryInvertedIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
                         index.getIndexType());
         }
-        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
@@ -181,9 +184,9 @@
 
     public abstract JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException;
 
-    public void setSourceLocation(SourceLocation sourceLoc) {
-        this.sourceLoc = sourceLoc;
-    }
+    protected abstract void setSecondaryRecDescAndComparators() throws AlgebricksException;
+
+    protected abstract int getNumSecondaryKeys();
 
     protected void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
@@ -218,7 +221,7 @@
         }
     }
 
-    protected void setFilterTypeTraitsAndComparators() throws AlgebricksException {
+    private void setFilterTypeTraitsAndComparators() throws AlgebricksException {
         filterTypeTraits = new ITypeTraits[numFilterFields];
         filterCmpFactories = new IBinaryComparatorFactory[numFilterFields];
         secondaryFilterFields = new int[numFilterFields];
@@ -239,9 +242,7 @@
         primaryFilterFields[0] = numPrimaryKeys + 1;
     }
 
-    protected abstract int getNumSecondaryKeys();
-
-    protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
+    private void setPrimaryRecDescAndComparators() throws AlgebricksException {
         List<List<String>> partitioningKeys = dataset.getPrimaryKeys();
         ISerializerDeserializer[] primaryRecFields =
                 new ISerializerDeserializer[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)];
@@ -271,8 +272,6 @@
         }
         primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
     }
-
-    protected abstract void setSecondaryRecDescAndComparators() throws AlgebricksException;
 
     protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec, int numSecondaryKeyFields,
             RecordDescriptor secondaryRecDesc) throws AlgebricksException {
@@ -349,16 +348,15 @@
         for (int i = 0; i < secondaryComparatorFactories.length; i++) {
             sortFields[i] = i;
         }
-        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
-                physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames, sortFields,
+                secondaryComparatorFactories, secondaryRecDesc);
         sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
 
     protected LSMIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws AlgebricksException {
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
         IndexDataflowHelperFactory primaryIndexDataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
 
@@ -372,8 +370,7 @@
     }
 
     protected TreeIndexBulkLoadOperatorDescriptor createExternalIndexBulkLoadOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws AlgebricksException {
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
         ExternalIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
                 ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider));
@@ -384,7 +381,7 @@
     }
 
     public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields,
-            RecordDescriptor secondaryRecDesc) throws AlgebricksException {
+            RecordDescriptor secondaryRecDesc) {
         IScalarEvaluatorFactory[] andArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
         NotDescriptor notDesc = new NotDescriptor();
         notDesc.setSourceLocation(sourceLoc);
@@ -454,7 +451,7 @@
     }
 
     protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec, int numSecondaryKeys,
-            RecordDescriptor secondaryRecDesc) throws AlgebricksException {
+            RecordDescriptor secondaryRecDesc) {
         int[] outColumns = new int[numSecondaryKeys];
         int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numSecondaryKeys; i++) {
@@ -477,8 +474,7 @@
     }
 
     protected ExternalIndexBulkModifyOperatorDescriptor createExternalIndexBulkModifyOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws AlgebricksException {
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
         // create a list of file ids
         int numOfDeletedFiles = 0;
         for (ExternalFile file : externalFiles) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 878aab6..afb562c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -46,6 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ShortPointable;
@@ -75,9 +75,9 @@
     private int[] invertedIndexFieldsForNonBulkLoadOps;
     private int[] secondaryFilterFieldsForNonBulkLoadOps;
 
-    protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -265,7 +265,7 @@
         return spec;
     }
 
-    private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
+    private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) {
         int docField = 0;
         int numSecondaryKeys = index.getKeyFieldNames().size();
         int[] primaryKeyFields = new int[numPrimaryKeys + numFilterFields];
@@ -289,14 +289,13 @@
         for (int i = 0; i < numTokenKeyPairFields; i++) {
             sortFields[i] = i;
         }
-        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
-                physOptConf.getMaxFramesExternalSort(), sortFields, tokenKeyPairComparatorFactories, secondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames, sortFields,
+                tokenKeyPairComparatorFactories, secondaryRecDesc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
 
-    private AbstractSingleActivityOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec)
-            throws AlgebricksException {
+    private AbstractSingleActivityOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
         int[] fieldPermutation = new int[numTokenKeyPairFields + numFilterFields];
         for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 1d6677e..1238389 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -38,7 +38,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
@@ -47,6 +46,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -67,9 +67,9 @@
     protected boolean isPointMBR;
     protected RecordDescriptor secondaryRecDescForPointMBR = null;
 
-    protected SecondaryRTreeOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryRTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -309,7 +309,7 @@
         return spec;
     }
 
-    protected int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
+    private int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
         int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
         int numSecondaryKeyFieldsForPointMBR = numSecondaryKeyFields / 2;
         int end = isPointMBR ? numSecondaryKeyFieldsForPointMBR : fieldPermutation.length;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index d78e80c..0373e32 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
@@ -47,9 +47,9 @@
 
 public abstract class SecondaryTreeIndexOperationsHelper extends SecondaryIndexOperationsHelper {
 
-    protected SecondaryTreeIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryTreeIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10168/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10173/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5073/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4606/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/119/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4694/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/124/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5262/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3770/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4916/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5081/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3775/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4653/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2697/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7521/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ali Alsuliman has submitted this change and it was merged.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


[ASTERIXDB-2474][COMP] Remove physical optimization static variable

- user model changes: no
- storage format changes: no
- interface changes: no

details:
This patch is to remove the physical optimization config static
variable used in IndexUtil to prevent side effects.

Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3020
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
13 files changed, 188 insertions(+), 170 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; No violations found; ; Verified
  Dmitry Lychagin: Looks good to me, approved



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 7b5eb83..98feafa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.algebra.operators.physical;
 
+import java.util.Map;
+
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSourceId;
@@ -111,7 +114,9 @@
             retainNull = true;
         }
         // In-memory budget (frame limit) for inverted-index search operations
-        int frameLimit = OptimizationConfUtil.getPhysicalOptimizationConfig().getMaxFramesForTextSearch();
+        CompilerProperties compilerProp = metadataProvider.getApplicationContext().getCompilerProperties();
+        Map<String, Object> queryConfig = metadataProvider.getConfig();
+        int frameLimit = OptimizationConfUtil.getTextSearchNumFrames(compilerProp, queryConfig, op.getSourceLocation());
 
         // Build runtime.
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch =
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 f9ad13f..8344d41 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
@@ -36,6 +36,7 @@
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.api.INodeJobTracker;
 import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -123,11 +124,6 @@
  */
 public class APIFramework {
 
-    private static final int MIN_FRAME_LIMIT_FOR_SORT = 3;
-    private static final int MIN_FRAME_LIMIT_FOR_GROUP_BY = 4;
-    private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
-    // one for query, two for intermediate results, one for final result, and one for reading an inverted list
-    private static final int MIN_FRAME_LIMIT_FOR_TEXTSEARCH = 5;
     private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
 
     // A white list of supported configurable parameters.
@@ -225,7 +221,7 @@
         CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
         Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
         final PhysicalOptimizationConfig physOptConf =
-                getPhysicalOptimizationConfig(compilerProperties, querySpecificConfig, sourceLoc);
+                OptimizationConfUtil.createPhysicalOptimizationConf(compilerProperties, querySpecificConfig, sourceLoc);
 
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
@@ -319,36 +315,6 @@
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
-    }
-
-    protected PhysicalOptimizationConfig getPhysicalOptimizationConfig(CompilerProperties compilerProperties,
-            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
-        int frameSize = compilerProperties.getFrameSize();
-        int sortFrameLimit = getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
-                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT, sourceLoc);
-        int groupFrameLimit = getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
-                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
-        int joinFrameLimit = getFrameLimit(CompilerProperties.COMPILER_JOINMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
-                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN, sourceLoc);
-        int textSearchFrameLimit = getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
-                (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
-                compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH, sourceLoc);
-        int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig);
-        boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
-
-        final PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
-        physOptConf.setFrameSize(frameSize);
-        physOptConf.setMaxFramesExternalSort(sortFrameLimit);
-        physOptConf.setMaxFramesExternalGroupBy(groupFrameLimit);
-        physOptConf.setMaxFramesForJoin(joinFrameLimit);
-        physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
-        physOptConf.setSortParallel(fullParallelSort);
-        physOptConf.setSortSamples(sortNumSamples);
-
-        return physOptConf;
     }
 
     protected IPrinterFactoryProvider getPrinterFactoryProvider(IDataFormat format,
@@ -475,43 +441,10 @@
         return ncMap.values().stream().mapToInt(NodeControllerInfo::getNumAvailableCores).sum();
     }
 
-    // Gets the frame limit.
-    private static int getFrameLimit(String parameterName, String parameter, long memBudgetInConfiguration,
-            int frameSize, int minFrameLimit, SourceLocation sourceLoc) throws AlgebricksException {
-        IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
-        long memBudget;
-        try {
-            memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
-        } catch (IllegalArgumentException e) {
-            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
-        }
-        int frameLimit = (int) (memBudget / frameSize);
-        if (frameLimit < minFrameLimit) {
-            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc, parameterName,
-                    frameSize * minFrameLimit);
-        }
-        // Sets the frame limit to the minimum frame limit if the caculated frame limit is too small.
-        return Math.max(frameLimit, minFrameLimit);
-    }
-
     // Gets the parallelism parameter.
     private static int getParallelism(String parameter, int parallelismInConfiguration) {
         IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.UNSIGNED_INTEGER;
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
-    }
-
-    private boolean getSortParallel(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
-        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_PARALLEL_KEY);
-        if (valueInQuery != null) {
-            return OptionTypes.BOOLEAN.parse(valueInQuery);
-        }
-        return compilerProperties.getSortParallel();
-    }
-
-    private int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
-        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
-        return valueInQuery == null ? compilerProperties.getSortSamples()
-                : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
     }
 
     // Validates if the query contains unsupported query parameters.
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 0471b25..9269b5e 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
@@ -18,13 +18,107 @@
  */
 package org.apache.asterix.common.config;
 
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.control.common.config.OptionTypes;
 
 public class OptimizationConfUtil {
 
-    private static final PhysicalOptimizationConfig physicalOptimizationConfig = new PhysicalOptimizationConfig();
+    private static final int MIN_FRAME_LIMIT_FOR_SORT = 3;
+    private static final int MIN_FRAME_LIMIT_FOR_GROUP_BY = 4;
+    private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
+    // one for query, two for intermediate results, one for final result, and one for reading an inverted list
+    private static final int MIN_FRAME_LIMIT_FOR_TEXT_SEARCH = 5;
 
-    public static PhysicalOptimizationConfig getPhysicalOptimizationConfig() {
-        return physicalOptimizationConfig;
+    private OptimizationConfUtil() {
+    }
+
+    public static PhysicalOptimizationConfig createPhysicalOptimizationConf(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+        int frameSize = compilerProperties.getFrameSize();
+        int sortFrameLimit = getSortNumFrames(compilerProperties, querySpecificConfig, sourceLoc);
+        int groupFrameLimit = getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
+                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
+        int joinFrameLimit = getFrameLimit(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
+                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN, sourceLoc);
+        int textSearchFrameLimit = getTextSearchNumFrames(compilerProperties, querySpecificConfig, sourceLoc);
+        int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig, sourceLoc);
+        boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
+
+        PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
+        physOptConf.setFrameSize(frameSize);
+        physOptConf.setMaxFramesExternalSort(sortFrameLimit);
+        physOptConf.setMaxFramesExternalGroupBy(groupFrameLimit);
+        physOptConf.setMaxFramesForJoin(joinFrameLimit);
+        physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
+        physOptConf.setSortParallel(fullParallelSort);
+        physOptConf.setSortSamples(sortNumSamples);
+
+        return physOptConf;
+    }
+
+    public static int getSortNumFrames(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        return getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
+                compilerProperties.getSortMemorySize(), compilerProperties.getFrameSize(), MIN_FRAME_LIMIT_FOR_SORT,
+                sourceLoc);
+    }
+
+    public static int getTextSearchNumFrames(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+        return getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
+                compilerProperties.getTextSearchMemorySize(), compilerProperties.getFrameSize(),
+                MIN_FRAME_LIMIT_FOR_TEXT_SEARCH, sourceLoc);
+    }
+
+    @SuppressWarnings("squid:S1166") // Either log or rethrow this exception
+    private static int getFrameLimit(String parameterName, String parameter, long memBudgetInConfiguration,
+            int frameSize, int minFrameLimit, SourceLocation sourceLoc) throws AlgebricksException {
+        IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
+        long memBudget;
+        try {
+            memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
+        }
+        int frameLimit = (int) (memBudget / frameSize);
+        if (frameLimit < minFrameLimit) {
+            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc, parameterName,
+                    frameSize * minFrameLimit);
+        }
+        // sets the frame limit to the minimum frame limit if the calculated frame limit is too small.
+        return Math.max(frameLimit, minFrameLimit);
+    }
+
+    private static boolean getSortParallel(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_PARALLEL_KEY);
+        if (valueInQuery != null) {
+            return OptionTypes.BOOLEAN.parse(valueInQuery);
+        }
+        return compilerProperties.getSortParallel();
+    }
+
+    @SuppressWarnings("squid:S1166") // Either log or rethrow this exception
+    private static int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig,
+            SourceLocation sourceLoc) throws AsterixException {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
+        try {
+            return valueInQuery == null ? compilerProperties.getSortSamples()
+                    : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc,
+                    CompilerProperties.COMPILER_SORT_SAMPLES_KEY, 1);
+        }
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index c29279b..311e4b6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -25,7 +25,6 @@
 import java.util.Set;
 
 import org.apache.asterix.common.config.DatasetConfig;
-import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.TxnId;
@@ -36,7 +35,6 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
@@ -46,8 +44,6 @@
 
     //TODO: replace this null with an empty array. currently, this breaks many tests
     private static final int[] empty = null;
-    private static final PhysicalOptimizationConfig physicalOptimizationConfig =
-            OptimizationConfUtil.getPhysicalOptimizationConfig();
 
     private IndexUtil() {
     }
@@ -105,22 +101,22 @@
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(options);
     }
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCreationJobSpec();
     }
 
@@ -135,10 +131,10 @@
         SecondaryIndexOperationsHelper secondaryIndexHelper;
         if (dataset.isCorrelated()) {
             secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
-                    index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+                    index, metadataProvider, sourceLoc);
         } else {
             secondaryIndexHelper = SecondaryTreeIndexOperationsHelper.createIndexOperationsHelper(dataset, index,
-                    metadataProvider, physicalOptimizationConfig, sourceLoc);
+                    metadataProvider, sourceLoc);
         }
         if (files != null) {
             secondaryIndexHelper.setExternalFiles(files);
@@ -148,8 +144,8 @@
 
     public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCompactJobSpec();
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 56e967e..36918d2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -33,7 +33,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
@@ -46,6 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -56,9 +56,9 @@
 
 public class SecondaryBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
 
-    protected SecondaryBTreeOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -292,7 +292,7 @@
 
     }
 
-    protected int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
+    private int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
         int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
         for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 12e21f6..3930563 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -31,7 +31,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
@@ -44,6 +43,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -56,9 +56,9 @@
  */
 public class SecondaryCorrelatedBTreeOperationsHelper extends SecondaryCorrelatedTreeIndexOperationsHelper {
 
-    protected SecondaryCorrelatedBTreeOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryCorrelatedBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index c6e4322..4699c82 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -33,7 +33,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -45,6 +44,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ShortPointable;
@@ -73,8 +73,8 @@
     private int[] secondaryFilterFieldsForNonBulkLoadOps;
 
     protected SecondaryCorrelatedInvertedIndexOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -305,9 +305,8 @@
         for (int i = 0; i < numPrimaryKeys; i++) {
             taggedSortFields[idx++] = i + numSecondaryKeys + NUM_TAG_FIELDS;
         }
-        ExternalSortOperatorDescriptor sortOp =
-                new ExternalSortOperatorDescriptor(spec, physOptConf.getMaxFramesExternalSort(), taggedSortFields,
-                        taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames,
+                taggedSortFields, taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index 7a855d5..c870c60 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -37,7 +37,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
@@ -46,6 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -61,9 +61,9 @@
     protected boolean isPointMBR;
     protected RecordDescriptor secondaryRecDescForPointMBR = null;
 
-    protected SecondaryCorrelatedRTreeOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryCorrelatedRTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index c8d0bf3..b044293 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -34,7 +34,6 @@
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
@@ -114,8 +113,8 @@
     }
 
     protected SecondaryCorrelatedTreeIndexOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     protected RecordDescriptor getTaggedRecordDescriptor(RecordDescriptor recDescriptor) {
@@ -267,9 +266,8 @@
         for (int i = 1; i < taggedSortFields.length; i++) {
             taggedSortFields[i] = i + 1;
         }
-        ExternalSortOperatorDescriptor sortOp =
-                new ExternalSortOperatorDescriptor(spec, physOptConf.getMaxFramesExternalSort(), taggedSortFields,
-                        taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames,
+                taggedSortFields, taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
         sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
@@ -295,7 +293,7 @@
     }
 
     protected IOperatorDescriptor createPrimaryIndexScanDiskComponentsOp(JobSpecification spec,
-            MetadataProvider metadataProvider, RecordDescriptor outRecDesc) throws AlgebricksException {
+            MetadataProvider metadataProvider, RecordDescriptor outRecDesc) {
         ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
         ISearchOperationCallbackFactory searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(
                 dataset.getDatasetId(), dataset.getPrimaryBloomFilterFields(), txnSubsystemProvider,
@@ -311,31 +309,29 @@
     }
 
     public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
-            throws AlgebricksException {
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case BTREE:
                 indexOperationsHelper =
-                        new SecondaryCorrelatedBTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                        new SecondaryCorrelatedBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case RTREE:
                 indexOperationsHelper =
-                        new SecondaryCorrelatedRTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                        new SecondaryCorrelatedRTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
                 indexOperationsHelper = new SecondaryCorrelatedInvertedIndexOperationsHelper(dataset, index,
-                        physOptConf, metadataProvider);
+                        metadataProvider, sourceLoc);
                 break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
                         index.getIndexType());
         }
-        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 5f9e6ef..7df723b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -26,6 +26,7 @@
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -55,7 +56,6 @@
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -82,7 +82,6 @@
 // possible index job specs,
 // not just for creation.
 public abstract class SecondaryIndexOperationsHelper {
-    protected final PhysicalOptimizationConfig physOptConf;
     protected final MetadataProvider metadataProvider;
     protected final Dataset dataset;
     protected final Index index;
@@ -119,14 +118,14 @@
     protected int[] secondaryBTreeFields;
     protected List<ExternalFile> externalFiles;
     protected int numPrimaryKeys;
-    protected SourceLocation sourceLoc;
+    protected final SourceLocation sourceLoc;
+    protected final int sortNumFrames;
 
     // Prevent public construction. Should be created via createIndexCreator().
-    protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
+    protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
         this.dataset = dataset;
         this.index = index;
-        this.physOptConf = physOptConf;
         this.metadataProvider = metadataProvider;
         this.itemType =
                 (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
@@ -134,6 +133,8 @@
         Pair<ARecordType, ARecordType> enforcedTypes = getEnforcedType(index, itemType, metaType);
         this.enforcedItemType = enforcedTypes.first;
         this.enforcedMetaType = enforcedTypes.second;
+        this.sourceLoc = sourceLoc;
+        this.sortNumFrames = getSortNumFrames(metadataProvider, sourceLoc);
     }
 
     private static Pair<ARecordType, ARecordType> getEnforcedType(Index index, ARecordType aRecordType,
@@ -143,32 +144,34 @@
                 : new Pair<>(null, null);
     }
 
-    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
+    private static int getSortNumFrames(MetadataProvider metadataProvider, SourceLocation sourceLoc)
             throws AlgebricksException {
+        return OptimizationConfUtil.getSortNumFrames(metadataProvider.getApplicationContext().getCompilerProperties(),
+                metadataProvider.getConfig(), sourceLoc);
+    }
+
+    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case BTREE:
-                indexOperationsHelper =
-                        new SecondaryBTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                indexOperationsHelper = new SecondaryBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case RTREE:
-                indexOperationsHelper =
-                        new SecondaryRTreeOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                indexOperationsHelper = new SecondaryRTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
                 indexOperationsHelper =
-                        new SecondaryInvertedIndexOperationsHelper(dataset, index, physOptConf, metadataProvider);
+                        new SecondaryInvertedIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
                         index.getIndexType());
         }
-        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
@@ -181,9 +184,9 @@
 
     public abstract JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException;
 
-    public void setSourceLocation(SourceLocation sourceLoc) {
-        this.sourceLoc = sourceLoc;
-    }
+    protected abstract void setSecondaryRecDescAndComparators() throws AlgebricksException;
+
+    protected abstract int getNumSecondaryKeys();
 
     protected void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
@@ -218,7 +221,7 @@
         }
     }
 
-    protected void setFilterTypeTraitsAndComparators() throws AlgebricksException {
+    private void setFilterTypeTraitsAndComparators() throws AlgebricksException {
         filterTypeTraits = new ITypeTraits[numFilterFields];
         filterCmpFactories = new IBinaryComparatorFactory[numFilterFields];
         secondaryFilterFields = new int[numFilterFields];
@@ -239,9 +242,7 @@
         primaryFilterFields[0] = numPrimaryKeys + 1;
     }
 
-    protected abstract int getNumSecondaryKeys();
-
-    protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
+    private void setPrimaryRecDescAndComparators() throws AlgebricksException {
         List<List<String>> partitioningKeys = dataset.getPrimaryKeys();
         ISerializerDeserializer[] primaryRecFields =
                 new ISerializerDeserializer[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)];
@@ -271,8 +272,6 @@
         }
         primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
     }
-
-    protected abstract void setSecondaryRecDescAndComparators() throws AlgebricksException;
 
     protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec, int numSecondaryKeyFields,
             RecordDescriptor secondaryRecDesc) throws AlgebricksException {
@@ -349,16 +348,15 @@
         for (int i = 0; i < secondaryComparatorFactories.length; i++) {
             sortFields[i] = i;
         }
-        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
-                physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames, sortFields,
+                secondaryComparatorFactories, secondaryRecDesc);
         sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
 
     protected LSMIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws AlgebricksException {
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
         IndexDataflowHelperFactory primaryIndexDataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
 
@@ -372,8 +370,7 @@
     }
 
     protected TreeIndexBulkLoadOperatorDescriptor createExternalIndexBulkLoadOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws AlgebricksException {
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
         ExternalIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
                 ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider));
@@ -384,7 +381,7 @@
     }
 
     public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields,
-            RecordDescriptor secondaryRecDesc) throws AlgebricksException {
+            RecordDescriptor secondaryRecDesc) {
         IScalarEvaluatorFactory[] andArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
         NotDescriptor notDesc = new NotDescriptor();
         notDesc.setSourceLocation(sourceLoc);
@@ -454,7 +451,7 @@
     }
 
     protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec, int numSecondaryKeys,
-            RecordDescriptor secondaryRecDesc) throws AlgebricksException {
+            RecordDescriptor secondaryRecDesc) {
         int[] outColumns = new int[numSecondaryKeys];
         int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numSecondaryKeys; i++) {
@@ -477,8 +474,7 @@
     }
 
     protected ExternalIndexBulkModifyOperatorDescriptor createExternalIndexBulkModifyOp(JobSpecification spec,
-            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws AlgebricksException {
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
         // create a list of file ids
         int numOfDeletedFiles = 0;
         for (ExternalFile file : externalFiles) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 878aab6..afb562c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -46,6 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ShortPointable;
@@ -75,9 +75,9 @@
     private int[] invertedIndexFieldsForNonBulkLoadOps;
     private int[] secondaryFilterFieldsForNonBulkLoadOps;
 
-    protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index,
-            PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -265,7 +265,7 @@
         return spec;
     }
 
-    private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
+    private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) {
         int docField = 0;
         int numSecondaryKeys = index.getKeyFieldNames().size();
         int[] primaryKeyFields = new int[numPrimaryKeys + numFilterFields];
@@ -289,14 +289,13 @@
         for (int i = 0; i < numTokenKeyPairFields; i++) {
             sortFields[i] = i;
         }
-        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
-                physOptConf.getMaxFramesExternalSort(), sortFields, tokenKeyPairComparatorFactories, secondaryRecDesc);
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec, sortNumFrames, sortFields,
+                tokenKeyPairComparatorFactories, secondaryRecDesc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
 
-    private AbstractSingleActivityOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec)
-            throws AlgebricksException {
+    private AbstractSingleActivityOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
         int[] fieldPermutation = new int[numTokenKeyPairFields + numFilterFields];
         for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 1d6677e..1238389 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -38,7 +38,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
@@ -47,6 +46,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -67,9 +67,9 @@
     protected boolean isPointMBR;
     protected RecordDescriptor secondaryRecDescForPointMBR = null;
 
-    protected SecondaryRTreeOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryRTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override
@@ -309,7 +309,7 @@
         return spec;
     }
 
-    protected int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
+    private int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
         int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
         int numSecondaryKeyFieldsForPointMBR = numSecondaryKeyFields / 2;
         int end = isPointMBR ? numSecondaryKeyFieldsForPointMBR : fieldPermutation.length;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index d78e80c..0373e32 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
@@ -47,9 +47,9 @@
 
 public abstract class SecondaryTreeIndexOperationsHelper extends SecondaryIndexOperationsHelper {
 
-    protected SecondaryTreeIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        super(dataset, index, physOptConf, metadataProvider);
+    protected SecondaryTreeIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
+        super(dataset, index, metadataProvider, sourceLoc);
     }
 
     @Override

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10176/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5078/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ali Alsuliman has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3020/1/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java:

Line 90:         } catch (IllegalArgumentException e) {
> Should we also catch NumberFormatException here? Seems like StorageUtil.get
IllegalArgumentException should take care of it, I believe. It's the super class.


Line 114:         try {
> POSITIVE_INTEGER.parse can throw IllegalArgumentException and NumberFormatE
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4686/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4913/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5028/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4658/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/122/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/3020

to look at the new patch set (#3).

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................

[ASTERIXDB-2474][COMP] Remove physical optimization static variable

- user model changes: no
- storage format changes: no
- interface changes: no

details:
This patch is to remove the physical optimization config static
variable used in IndexUtil to prevent side effects.

Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
13 files changed, 188 insertions(+), 170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/20/3020/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4661/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8643/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7518/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5257/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/zXFzQh : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/XoHxuk : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://goo.gl/gf8dKW : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7518/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4697/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4689/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7521/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8640/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3775/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8635/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4598/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/vzv6cx : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/127/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/J8f5VA : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2705/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7513/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/117/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4603/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4694/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5033/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4691/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3020/1/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java:

Line 90:         } catch (IllegalArgumentException e) {
> CRITICAL SonarQube violation:
Should we also catch NumberFormatException here? Seems like StorageUtil.getSizeInBytes can throw it as well.


Line 114:                 : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
POSITIVE_INTEGER.parse can throw IllegalArgumentException and NumberFormatException. Should we catch them here?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2702/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3770/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5265/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4908/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5036/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/125/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/3020

to look at the new patch set (#2).

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................

[ASTERIXDB-2474][COMP] Remove physical optimization static variable

- user model changes: no
- storage format changes: no
- interface changes: no

details:
This patch is to remove the physical optimization config static
variable used in IndexUtil to prevent side effects.

Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
13 files changed, 186 insertions(+), 170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/20/3020/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7513/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2474][COMP] Remove physical optimization static v...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2474][COMP] Remove physical optimization static variable
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/P9jJyP : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3020
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I83f58715e5d643fa5b52f6e0a0d1ad1cb839f895
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No