Merge branch 'master' into raman/master_feeds
diff --git a/.gitignore b/.gitignore
index babae45..503b769 100644
--- a/.gitignore
+++ b/.gitignore
@@ -10,6 +10,8 @@
expected
teststore1
teststore2
+dev1
+dev2
derby.log
hadoop-conf-tmp
metastore_db
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index af65ebd..c9ef2f3 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -57,8 +57,8 @@
INSERT_DELETE,
INDEX_INSERT_DELETE,
UPDATE,
- INVERTED_INDEX_SEARCH,
- FUZZY_INVERTED_INDEX_SEARCH,
+ SINGLE_PARTITION_INVERTED_INDEX_SEARCH,
+ LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
PARTITIONINGSPLIT,
EXTENSION_OPERATOR
}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
index 12a5282..703bc1f 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
@@ -18,7 +18,8 @@
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IPushRuntimeFactory extends Serializable {
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException;
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException, HyracksDataException;
}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index c17b06f..8f2eaac 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -141,7 +141,7 @@
}
private IFrameWriter assemblePipeline(AlgebricksPipeline subplan, IFrameWriter writer, IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws AlgebricksException, HyracksDataException {
// plug the operators
IFrameWriter start = writer;
IPushRuntimeFactory[] runtimeFactories = subplan.getRuntimeFactories();
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index 7f5fead..082e98a 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -85,7 +85,7 @@
}
}
- protected final void initAccessAppend(IHyracksTaskContext ctx) {
+ protected final void initAccessAppend(IHyracksTaskContext ctx) throws HyracksDataException {
// if (allocFrame) {
frame = ctx.allocateFrame();
appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -94,7 +94,7 @@
tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
}
- protected final void initAccessAppendRef(IHyracksTaskContext ctx) {
+ protected final void initAccessAppendRef(IHyracksTaskContext ctx) throws HyracksDataException {
initAccessAppend(ctx);
tRef = new FrameTupleReference();
}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
index 084b085..43270b6 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public abstract class AbstractOneInputOneOutputRuntimeFactory implements IPushRuntimeFactory {
@@ -30,11 +31,11 @@
}
@Override
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException, HyracksDataException {
return createOneOutputPushRuntime(ctx);
}
public abstract AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx)
- throws AlgebricksException;
+ throws AlgebricksException, HyracksDataException;
}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
index cef05d8..5bcf933 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public class PipelineAssembler {
@@ -40,7 +41,8 @@
this.outputArity = outputArity;
}
- public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws AlgebricksException {
+ public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws AlgebricksException,
+ HyracksDataException {
// plug the operators
IFrameWriter start = writer;// this.writer;
for (int i = pipeline.getRuntimeFactories().length - 1; i >= 0; i--) {
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 6e43bd1..0499684 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -67,7 +67,7 @@
@Override
public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws AlgebricksException, HyracksDataException {
RecordDescriptor pipelineOutputRecordDescriptor = null;
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
index 38eddd1..745fdf6 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
@@ -38,7 +38,7 @@
}
@Override
- public IPushRuntime createPushRuntime(final IHyracksTaskContext ctx) {
+ public IPushRuntime createPushRuntime(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractOneInputSourcePushRuntime() {
private ByteBuffer frame = ctx.allocateFrame();
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
index dc155a2..5486fae 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -35,13 +35,13 @@
}
@Override
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) {
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
return new NestedTupleSourceRuntime(ctx);
}
public static class NestedTupleSourceRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
- public NestedTupleSourceRuntime(IHyracksTaskContext ctx) {
+ public NestedTupleSourceRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
initAccessAppend(ctx);
}
diff --git a/hivesterix/hivesterix-dist/pom.xml b/hivesterix/hivesterix-dist/pom.xml
index 2adcc7c..8ecdfe1 100644
--- a/hivesterix/hivesterix-dist/pom.xml
+++ b/hivesterix/hivesterix-dist/pom.xml
@@ -14,9 +14,7 @@
!-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hivesterix-dist</artifactId>
- <version>0.2.7-SNAPSHOT</version>
<name>hivesterix-dist</name>
<parent>
diff --git a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
index 0c2416d..d3bcaca 100644
--- a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
+++ b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -12,598 +12,601 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hivesterix.runtime.exec;
-
-import java.io.BufferedReader;
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
-import java.io.Serializable;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.exec.ConditionalTask;
-import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
-import org.apache.hadoop.hive.ql.exec.MapRedTask;
-import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.TableScanOperator;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
-import org.apache.hadoop.hive.ql.plan.FetchWork;
-import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
-import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
-import org.apache.hadoop.hive.ql.plan.MapredWork;
-import org.apache.hadoop.hive.ql.plan.PartitionDesc;
-import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-
-import edu.uci.ics.hivesterix.common.config.ConfUtil;
-import edu.uci.ics.hivesterix.logical.expression.HiveExpressionTypeComputer;
-import edu.uci.ics.hivesterix.logical.expression.HiveMergeAggregationExpressionFactory;
-import edu.uci.ics.hivesterix.logical.expression.HiveNullableTypeComputer;
-import edu.uci.ics.hivesterix.logical.expression.HivePartialAggregationTypeComputer;
-import edu.uci.ics.hivesterix.logical.plan.HiveAlgebricksTranslator;
-import edu.uci.ics.hivesterix.logical.plan.HiveLogicalPlanAndMetaData;
-import edu.uci.ics.hivesterix.optimizer.rulecollections.HiveRuleCollections;
-import edu.uci.ics.hivesterix.runtime.factory.evaluator.HiveExpressionRuntimeProvider;
-import edu.uci.ics.hivesterix.runtime.factory.nullwriter.HiveNullWriterFactory;
-import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryBooleanInspectorFactory;
-import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryIntegerInspectorFactory;
-import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy;
-import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy.Policy;
-import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryComparatorFactoryProvider;
-import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFactoryProvider;
-import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFamilyProvider;
-import edu.uci.ics.hivesterix.runtime.provider.HiveNormalizedKeyComputerFactoryProvider;
-import edu.uci.ics.hivesterix.runtime.provider.HivePrinterFactoryProvider;
-import edu.uci.ics.hivesterix.runtime.provider.HiveSerializerDeserializerProvider;
-import edu.uci.ics.hivesterix.runtime.provider.HiveTypeTraitProvider;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
-import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder.DefaultOptimizationContextFactory;
-import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
-import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
-import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
-import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class HyracksExecutionEngine implements IExecutionEngine {
-
- private static final Log LOG = LogFactory.getLog(HyracksExecutionEngine.class.getName());
- private static final String clusterPropertiesPath = "conf/cluster.properties";
- private static final String masterFilePath = "conf/master";
-
- private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_LOGICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
- private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_PHYSICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
- static {
- SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
- SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
- SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
- DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
- HiveRuleCollections.NORMALIZATION));
- DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
- HiveRuleCollections.COND_PUSHDOWN_AND_JOIN_INFERENCE));
- DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
- HiveRuleCollections.LOAD_FIELDS));
- DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
- HiveRuleCollections.OP_PUSHDOWN));
- DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
- HiveRuleCollections.DATA_EXCHANGE));
- DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
- HiveRuleCollections.CONSOLIDATION));
-
- DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
- HiveRuleCollections.PHYSICAL_PLAN_REWRITES));
- DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
- HiveRuleCollections.prepareJobGenRules));
- }
-
- /**
- * static configurations for compiler
- */
- private HeuristicCompilerFactoryBuilder builder;
-
- /**
- * compiler
- */
- private ICompiler compiler;
-
- /**
- * physical optimization config
- */
- private PhysicalOptimizationConfig physicalOptimizationConfig;
-
- /**
- * final ending operators
- */
- private List<Operator> leaveOps = new ArrayList<Operator>();
-
- /**
- * tasks that are already visited
- */
- private Map<Task<? extends Serializable>, Boolean> tasksVisited = new HashMap<Task<? extends Serializable>, Boolean>();
-
- /**
- * hyracks job spec
- */
- private JobSpecification jobSpec;
-
- /**
- * hive configuration
- */
- private HiveConf conf;
-
- /**
- * plan printer
- */
- private PrintWriter planPrinter;
-
- /**
- * properties
- */
- private Properties clusterProps;
-
- /**
- * the Hyracks client connection
- */
- private IHyracksClientConnection hcc;
-
- public HyracksExecutionEngine(HiveConf conf) {
- this.conf = conf;
- init(conf);
- }
-
- public HyracksExecutionEngine(HiveConf conf, PrintWriter planPrinter) {
- this.conf = conf;
- this.planPrinter = planPrinter;
- init(conf);
- }
-
- private void init(HiveConf conf) {
- builder = new HeuristicCompilerFactoryBuilder(DefaultOptimizationContextFactory.INSTANCE);
- builder.setLogicalRewrites(DEFAULT_LOGICAL_REWRITES);
- builder.setPhysicalRewrites(DEFAULT_PHYSICAL_REWRITES);
- builder.setIMergeAggregationExpressionFactory(HiveMergeAggregationExpressionFactory.INSTANCE);
- builder.setExpressionTypeComputer(HiveExpressionTypeComputer.INSTANCE);
- builder.setNullableTypeComputer(HiveNullableTypeComputer.INSTANCE);
-
- long memSizeExternalGby = conf.getLong("hive.algebricks.groupby.external.memory", 268435456);
- long memSizeExternalSort = conf.getLong("hive.algebricks.sort.memory", 536870912);
- int frameSize = conf.getInt("hive.algebricks.framesize", 32768);
-
- physicalOptimizationConfig = new PhysicalOptimizationConfig();
- int frameLimitExtGby = (int) (memSizeExternalGby / frameSize);
- physicalOptimizationConfig.setMaxFramesExternalGroupBy(frameLimitExtGby);
- int frameLimitExtSort = (int) (memSizeExternalSort / frameSize);
- physicalOptimizationConfig.setMaxFramesExternalSort(frameLimitExtSort);
- builder.setPhysicalOptimizationConfig(physicalOptimizationConfig);
- }
-
- @Override
- public int compileJob(List<Task<? extends Serializable>> rootTasks) {
- // clean up
- leaveOps.clear();
- tasksVisited.clear();
- jobSpec = null;
-
- HashMap<String, PartitionDesc> aliasToPath = new HashMap<String, PartitionDesc>();
- List<Operator> rootOps = generateRootOperatorDAG(rootTasks, aliasToPath);
-
- // get all leave Ops
- getLeaves(rootOps, leaveOps);
-
- HiveAlgebricksTranslator translator = new HiveAlgebricksTranslator();
- try {
- translator.translate(rootOps, null, aliasToPath);
-
- ILogicalPlan plan = translator.genLogicalPlan();
-
- if (plan.getRoots() != null && plan.getRoots().size() > 0 && plan.getRoots().get(0).getValue() != null) {
- translator.printOperators();
- ILogicalPlanAndMetadata planAndMetadata = new HiveLogicalPlanAndMetaData(plan,
- translator.getMetadataProvider());
-
- ICompilerFactory compilerFactory = builder.create();
- compiler = compilerFactory.createCompiler(planAndMetadata.getPlan(),
- planAndMetadata.getMetadataProvider(), translator.getVariableCounter());
-
- // run optimization and re-writing rules for Hive plan
- compiler.optimize();
-
- // print optimized plan
- LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
- StringBuilder buffer = new StringBuilder();
- PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
- String planStr = buffer.toString();
- System.out.println(planStr);
-
- if (planPrinter != null)
- planPrinter.print(planStr);
- }
- } catch (Exception e) {
- e.printStackTrace();
- return 1;
- }
-
- return 0;
- }
-
- private void codeGen() throws AlgebricksException {
- try {
- // number of cpu cores in the cluster
- builder.setClusterLocations(new AlgebricksAbsolutePartitionConstraint(ConfUtil.getNCs()));
- } catch (Exception e) {
- throw new AlgebricksException(e);
- }
- // builder.setClusterTopology(ConfUtil.getClusterTopology());
- builder.setBinaryBooleanInspectorFactory(HiveBinaryBooleanInspectorFactory.INSTANCE);
- builder.setBinaryIntegerInspectorFactory(HiveBinaryIntegerInspectorFactory.INSTANCE);
- builder.setComparatorFactoryProvider(HiveBinaryComparatorFactoryProvider.INSTANCE);
- builder.setExpressionRuntimeProvider(HiveExpressionRuntimeProvider.INSTANCE);
- builder.setHashFunctionFactoryProvider(HiveBinaryHashFunctionFactoryProvider.INSTANCE);
- builder.setPrinterProvider(HivePrinterFactoryProvider.INSTANCE);
- builder.setSerializerDeserializerProvider(HiveSerializerDeserializerProvider.INSTANCE);
- builder.setNullWriterFactory(HiveNullWriterFactory.INSTANCE);
- builder.setNormalizedKeyComputerFactoryProvider(HiveNormalizedKeyComputerFactoryProvider.INSTANCE);
- builder.setPartialAggregationTypeComputer(HivePartialAggregationTypeComputer.INSTANCE);
- builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);
- builder.setHashFunctionFamilyProvider(HiveBinaryHashFunctionFamilyProvider.INSTANCE);
-
- jobSpec = compiler.createJob(null, null);
-
- // set the policy
- String policyStr = conf.get("hive.hyracks.connectorpolicy");
- if (policyStr == null)
- policyStr = "PIPELINING";
- Policy policyValue = Policy.valueOf(policyStr);
- jobSpec.setConnectorPolicyAssignmentPolicy(new HiveConnectorPolicyAssignmentPolicy(policyValue));
- jobSpec.setUseConnectorPolicyForScheduling(false);
- }
-
- @Override
- public int executeJob() {
- try {
- codeGen();
- executeHyracksJob(jobSpec);
- } catch (Exception e) {
- e.printStackTrace();
- return 1;
- }
- return 0;
- }
-
- private List<Operator> generateRootOperatorDAG(List<Task<? extends Serializable>> rootTasks,
- HashMap<String, PartitionDesc> aliasToPath) {
-
- List<Operator> rootOps = new ArrayList<Operator>();
- List<Task<? extends Serializable>> toDelete = new ArrayList<Task<? extends Serializable>>();
- tasksVisited.clear();
-
- for (int i = rootTasks.size() - 1; i >= 0; i--) {
- /**
- * list of map-reduce tasks
- */
- Task<? extends Serializable> task = rootTasks.get(i);
-
- if (task instanceof MapRedTask) {
- List<Operator> mapRootOps = articulateMapReduceOperators(task, rootOps, aliasToPath, rootTasks);
- if (i == 0)
- rootOps.addAll(mapRootOps);
- else {
- List<Operator> leaves = new ArrayList<Operator>();
- getLeaves(rootOps, leaves);
-
- List<Operator> mapChildren = new ArrayList<Operator>();
- for (Operator childMap : mapRootOps) {
- if (childMap instanceof TableScanOperator) {
- TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
- if (topDesc == null)
- mapChildren.add(childMap);
- else {
- rootOps.add(childMap);
- }
- } else
- mapChildren.add(childMap);
- }
-
- if (mapChildren.size() > 0) {
- for (Operator leaf : leaves)
- leaf.setChildOperators(mapChildren);
- for (Operator child : mapChildren)
- child.setParentOperators(leaves);
- }
- }
-
- MapredWork mr = (MapredWork) task.getWork();
- HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
-
- addAliasToPartition(aliasToPath, map);
- toDelete.add(task);
- }
- }
-
- for (Task<? extends Serializable> task : toDelete)
- rootTasks.remove(task);
-
- return rootOps;
- }
-
- private void addAliasToPartition(HashMap<String, PartitionDesc> aliasToPath, HashMap<String, PartitionDesc> map) {
- Iterator<String> keys = map.keySet().iterator();
- while (keys.hasNext()) {
- String key = keys.next();
- PartitionDesc part = map.get(key);
- String[] names = key.split(":");
- for (String name : names) {
- aliasToPath.put(name, part);
- }
- }
- }
-
- private List<Operator> articulateMapReduceOperators(Task task, List<Operator> rootOps,
- HashMap<String, PartitionDesc> aliasToPath, List<Task<? extends Serializable>> rootTasks) {
- // System.out.println("!"+task.getName());
- if (!(task instanceof MapRedTask)) {
- if (!(task instanceof ConditionalTask)) {
- rootTasks.add(task);
- return null;
- } else {
- // remove map-reduce branches in condition task
- ConditionalTask condition = (ConditionalTask) task;
- List<Task<? extends Serializable>> branches = condition.getListTasks();
- for (int i = branches.size() - 1; i >= 0; i--) {
- Task branch = branches.get(i);
- if (branch instanceof MapRedTask) {
- return articulateMapReduceOperators(branch, rootOps, aliasToPath, rootTasks);
- }
- }
- rootTasks.add(task);
- return null;
- }
- }
-
- MapredWork mr = (MapredWork) task.getWork();
- HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
-
- // put all aliasToParitionDesc mapping into the map
- addAliasToPartition(aliasToPath, map);
-
- MapRedTask mrtask = (MapRedTask) task;
- MapredWork work = (MapredWork) mrtask.getWork();
- HashMap<String, Operator<? extends Serializable>> operators = work.getAliasToWork();
-
- Set entries = operators.entrySet();
- Iterator<Entry<String, Operator>> iterator = entries.iterator();
- List<Operator> mapRootOps = new ArrayList<Operator>();
-
- // get map root operators
- while (iterator.hasNext()) {
- Operator next = iterator.next().getValue();
- if (!mapRootOps.contains(next)) {
- // clear that only for the case of union
- mapRootOps.add(next);
- }
- }
-
- // get map local work
- MapredLocalWork localWork = work.getMapLocalWork();
- if (localWork != null) {
- HashMap<String, Operator<? extends Serializable>> localOperators = localWork.getAliasToWork();
-
- Set localEntries = localOperators.entrySet();
- Iterator<Entry<String, Operator>> localIterator = localEntries.iterator();
- while (localIterator.hasNext()) {
- mapRootOps.add(localIterator.next().getValue());
- }
-
- HashMap<String, FetchWork> localFetch = localWork.getAliasToFetchWork();
- Set localFetchEntries = localFetch.entrySet();
- Iterator<Entry<String, FetchWork>> localFetchIterator = localFetchEntries.iterator();
- while (localFetchIterator.hasNext()) {
- Entry<String, FetchWork> fetchMap = localFetchIterator.next();
- FetchWork fetch = fetchMap.getValue();
- String alias = fetchMap.getKey();
- List<PartitionDesc> dirPart = fetch.getPartDesc();
-
- // temporary hack: put the first partitionDesc into the map
- aliasToPath.put(alias, dirPart.get(0));
- }
- }
-
- Boolean visited = tasksVisited.get(task);
- if (visited != null && visited.booleanValue() == true) {
- return mapRootOps;
- }
-
- // do that only for union operator
- for (Operator op : mapRootOps)
- if (op.getParentOperators() != null)
- op.getParentOperators().clear();
-
- List<Operator> mapLeaves = new ArrayList<Operator>();
- downToLeaves(mapRootOps, mapLeaves);
- List<Operator> reduceOps = new ArrayList<Operator>();
-
- if (work.getReducer() != null)
- reduceOps.add(work.getReducer());
-
- for (Operator mapLeaf : mapLeaves) {
- mapLeaf.setChildOperators(reduceOps);
- }
-
- for (Operator reduceOp : reduceOps) {
- if (reduceOp != null)
- reduceOp.setParentOperators(mapLeaves);
- }
-
- List<Operator> leafs = new ArrayList<Operator>();
- if (reduceOps.size() > 0) {
- downToLeaves(reduceOps, leafs);
- } else {
- leafs = mapLeaves;
- }
-
- List<Operator> mapChildren = new ArrayList<Operator>();
- if (task.getChildTasks() != null && task.getChildTasks().size() > 0) {
- for (Object child : task.getChildTasks()) {
- List<Operator> childMapOps = articulateMapReduceOperators((Task) child, rootOps, aliasToPath, rootTasks);
- if (childMapOps == null)
- continue;
-
- for (Operator childMap : childMapOps) {
- if (childMap instanceof TableScanOperator) {
- TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
- if (topDesc == null)
- mapChildren.add(childMap);
- else {
- rootOps.add(childMap);
- }
- } else {
- // if not table scan, add the child
- mapChildren.add(childMap);
- }
- }
- }
-
- if (mapChildren.size() > 0) {
- int i = 0;
- for (Operator leaf : leafs) {
- if (leaf.getChildOperators() == null || leaf.getChildOperators().size() == 0)
- leaf.setChildOperators(new ArrayList<Operator>());
- leaf.getChildOperators().add(mapChildren.get(i));
- i++;
- }
- i = 0;
- for (Operator child : mapChildren) {
- if (child.getParentOperators() == null || child.getParentOperators().size() == 0)
- child.setParentOperators(new ArrayList<Operator>());
- child.getParentOperators().add(leafs.get(i));
- i++;
- }
- }
- }
-
- // mark this task as visited
- this.tasksVisited.put(task, true);
- return mapRootOps;
- }
-
- /**
- * down to leaf nodes
- *
- * @param ops
- * @param leaves
- */
- private void downToLeaves(List<Operator> ops, List<Operator> leaves) {
-
- // Operator currentOp;
- for (Operator op : ops) {
- if (op != null && op.getChildOperators() != null && op.getChildOperators().size() > 0) {
- downToLeaves(op.getChildOperators(), leaves);
- } else {
- if (op != null && leaves.indexOf(op) < 0)
- leaves.add(op);
- }
- }
- }
-
- private void getLeaves(List<Operator> roots, List<Operator> currentLeaves) {
- for (Operator op : roots) {
- List<Operator> children = op.getChildOperators();
- if (children == null || children.size() <= 0) {
- currentLeaves.add(op);
- } else {
- getLeaves(children, currentLeaves);
- }
- }
- }
-
- private void executeHyracksJob(JobSpecification job) throws Exception {
-
- /**
- * load the properties file if it is not loaded
- */
- if (clusterProps == null) {
- clusterProps = new Properties();
- InputStream confIn = new FileInputStream(clusterPropertiesPath);
- clusterProps.load(confIn);
- confIn.close();
- }
-
- if (hcc == null) {
- BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));
- String masterNode = ipReader.readLine();
- ipReader.close();
-
- InetAddress[] ips = InetAddress.getAllByName(masterNode);
- int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));
- for (InetAddress ip : ips) {
- if (ip.getAddress().length <= 4) {
- try {
- hcc = new HyracksConnection(ip.getHostAddress(), port);
- break;
- } catch (Exception e) {
- continue;
- }
- }
- }
- }
-
- long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(job);
- hcc.waitForCompletion(jobId);
-
- // System.out.println("job finished: " + jobId.toString());
- // call all leave nodes to end
- for (Operator leaf : leaveOps) {
- jobClose(leaf);
- }
-
- long end = System.currentTimeMillis();
- System.err.println(start + " " + end + " " + (end - start));
- }
-
- /**
- * mv to final directory on hdfs (not real final)
- *
- * @param leaf
- * @throws Exception
- */
- private void jobClose(Operator leaf) throws Exception {
- FileSinkOperator fsOp = (FileSinkOperator) leaf;
- FileSinkDesc desc = fsOp.getConf();
- boolean isNativeTable = !desc.getTableInfo().isNonNative();
- if ((conf != null) && isNativeTable) {
- String specPath = desc.getDirName();
- DynamicPartitionCtx dpCtx = desc.getDynPartCtx();
- // for 0.7.0
- fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);
- // for 0.8.0
- // Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx,
- // desc);
- }
- }
-}
+package edu.uci.ics.hivesterix.runtime.exec;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.MapRedTask;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
+import org.apache.hadoop.hive.ql.plan.FetchWork;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+
+import edu.uci.ics.hivesterix.common.config.ConfUtil;
+import edu.uci.ics.hivesterix.logical.expression.HiveExpressionTypeComputer;
+import edu.uci.ics.hivesterix.logical.expression.HiveMergeAggregationExpressionFactory;
+import edu.uci.ics.hivesterix.logical.expression.HiveNullableTypeComputer;
+import edu.uci.ics.hivesterix.logical.expression.HivePartialAggregationTypeComputer;
+import edu.uci.ics.hivesterix.logical.plan.HiveAlgebricksTranslator;
+import edu.uci.ics.hivesterix.logical.plan.HiveLogicalPlanAndMetaData;
+import edu.uci.ics.hivesterix.optimizer.rulecollections.HiveRuleCollections;
+import edu.uci.ics.hivesterix.runtime.factory.evaluator.HiveExpressionRuntimeProvider;
+import edu.uci.ics.hivesterix.runtime.factory.nullwriter.HiveNullWriterFactory;
+import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryBooleanInspectorFactory;
+import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryIntegerInspectorFactory;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy.Policy;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryComparatorFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFamilyProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveNormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HivePrinterFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveSerializerDeserializerProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveTypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder.DefaultOptimizationContextFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class HyracksExecutionEngine implements IExecutionEngine {
+
+ private static final Log LOG = LogFactory.getLog(HyracksExecutionEngine.class.getName());
+ private static final String clusterPropertiesPath = "conf/cluster.properties";
+ private static final String masterFilePath = "conf/master";
+
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_LOGICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_PHYSICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+ static {
+ SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+ SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
+ SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+ HiveRuleCollections.NORMALIZATION));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ HiveRuleCollections.COND_PUSHDOWN_AND_JOIN_INFERENCE));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+ HiveRuleCollections.LOAD_FIELDS));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ HiveRuleCollections.OP_PUSHDOWN));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ HiveRuleCollections.DATA_EXCHANGE));
+ DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ HiveRuleCollections.CONSOLIDATION));
+
+ DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ HiveRuleCollections.PHYSICAL_PLAN_REWRITES));
+ DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ HiveRuleCollections.prepareJobGenRules));
+ }
+
+ /**
+ * static configurations for compiler
+ */
+ private HeuristicCompilerFactoryBuilder builder;
+
+ /**
+ * compiler
+ */
+ private ICompiler compiler;
+
+ /**
+ * physical optimization config
+ */
+ private PhysicalOptimizationConfig physicalOptimizationConfig;
+
+ /**
+ * final ending operators
+ */
+ private List<Operator> leaveOps = new ArrayList<Operator>();
+
+ /**
+ * tasks that are already visited
+ */
+ private Map<Task<? extends Serializable>, Boolean> tasksVisited = new HashMap<Task<? extends Serializable>, Boolean>();
+
+ /**
+ * hyracks job spec
+ */
+ private JobSpecification jobSpec;
+
+ /**
+ * hive configuration
+ */
+ private HiveConf conf;
+
+ /**
+ * plan printer
+ */
+ private PrintWriter planPrinter;
+
+ /**
+ * properties
+ */
+ private Properties clusterProps;
+
+ /**
+ * the Hyracks client connection
+ */
+ private IHyracksClientConnection hcc;
+
+ public HyracksExecutionEngine(HiveConf conf) {
+ this.conf = conf;
+ init(conf);
+ }
+
+ public HyracksExecutionEngine(HiveConf conf, PrintWriter planPrinter) {
+ this.conf = conf;
+ this.planPrinter = planPrinter;
+ init(conf);
+ }
+
+ private void init(HiveConf conf) {
+ builder = new HeuristicCompilerFactoryBuilder(DefaultOptimizationContextFactory.INSTANCE);
+ builder.setLogicalRewrites(DEFAULT_LOGICAL_REWRITES);
+ builder.setPhysicalRewrites(DEFAULT_PHYSICAL_REWRITES);
+ builder.setIMergeAggregationExpressionFactory(HiveMergeAggregationExpressionFactory.INSTANCE);
+ builder.setExpressionTypeComputer(HiveExpressionTypeComputer.INSTANCE);
+ builder.setNullableTypeComputer(HiveNullableTypeComputer.INSTANCE);
+
+ long memSizeExternalGby = conf.getLong("hive.algebricks.groupby.external.memory", 268435456);
+ long memSizeExternalSort = conf.getLong("hive.algebricks.sort.memory", 536870912);
+ int frameSize = conf.getInt("hive.algebricks.framesize", 32768);
+
+ physicalOptimizationConfig = new PhysicalOptimizationConfig();
+ int frameLimitExtGby = (int) (memSizeExternalGby / frameSize);
+ physicalOptimizationConfig.setMaxFramesExternalGroupBy(frameLimitExtGby);
+ int frameLimitExtSort = (int) (memSizeExternalSort / frameSize);
+ physicalOptimizationConfig.setMaxFramesExternalSort(frameLimitExtSort);
+ builder.setPhysicalOptimizationConfig(physicalOptimizationConfig);
+ }
+
+ @Override
+ public int compileJob(List<Task<? extends Serializable>> rootTasks) {
+ // clean up
+ leaveOps.clear();
+ tasksVisited.clear();
+ jobSpec = null;
+
+ HashMap<String, PartitionDesc> aliasToPath = new HashMap<String, PartitionDesc>();
+ List<Operator> rootOps = generateRootOperatorDAG(rootTasks, aliasToPath);
+
+ // get all leave Ops
+ getLeaves(rootOps, leaveOps);
+
+ HiveAlgebricksTranslator translator = new HiveAlgebricksTranslator();
+ try {
+ translator.translate(rootOps, null, aliasToPath);
+
+ ILogicalPlan plan = translator.genLogicalPlan();
+
+ if (plan.getRoots() != null && plan.getRoots().size() > 0 && plan.getRoots().get(0).getValue() != null) {
+ translator.printOperators();
+ ILogicalPlanAndMetadata planAndMetadata = new HiveLogicalPlanAndMetaData(plan,
+ translator.getMetadataProvider());
+
+ ICompilerFactory compilerFactory = builder.create();
+ compiler = compilerFactory.createCompiler(planAndMetadata.getPlan(),
+ planAndMetadata.getMetadataProvider(), translator.getVariableCounter());
+
+ // run optimization and re-writing rules for Hive plan
+ compiler.optimize();
+
+ // print optimized plan
+ LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+ StringBuilder buffer = new StringBuilder();
+ PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
+ String planStr = buffer.toString();
+ System.out.println(planStr);
+
+ if (planPrinter != null)
+ planPrinter.print(planStr);
+ } else {
+ /** it is not a map reduce task DAG */
+ return 2;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ return 1;
+ }
+
+ return 0;
+ }
+
+ private void codeGen() throws AlgebricksException {
+ try {
+ // number of cpu cores in the cluster
+ builder.setClusterLocations(new AlgebricksAbsolutePartitionConstraint(ConfUtil.getNCs()));
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ // builder.setClusterTopology(ConfUtil.getClusterTopology());
+ builder.setBinaryBooleanInspectorFactory(HiveBinaryBooleanInspectorFactory.INSTANCE);
+ builder.setBinaryIntegerInspectorFactory(HiveBinaryIntegerInspectorFactory.INSTANCE);
+ builder.setComparatorFactoryProvider(HiveBinaryComparatorFactoryProvider.INSTANCE);
+ builder.setExpressionRuntimeProvider(HiveExpressionRuntimeProvider.INSTANCE);
+ builder.setHashFunctionFactoryProvider(HiveBinaryHashFunctionFactoryProvider.INSTANCE);
+ builder.setPrinterProvider(HivePrinterFactoryProvider.INSTANCE);
+ builder.setSerializerDeserializerProvider(HiveSerializerDeserializerProvider.INSTANCE);
+ builder.setNullWriterFactory(HiveNullWriterFactory.INSTANCE);
+ builder.setNormalizedKeyComputerFactoryProvider(HiveNormalizedKeyComputerFactoryProvider.INSTANCE);
+ builder.setPartialAggregationTypeComputer(HivePartialAggregationTypeComputer.INSTANCE);
+ builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);
+ builder.setHashFunctionFamilyProvider(HiveBinaryHashFunctionFamilyProvider.INSTANCE);
+
+ jobSpec = compiler.createJob(null, null);
+
+ // set the policy
+ String policyStr = conf.get("hive.hyracks.connectorpolicy");
+ if (policyStr == null)
+ policyStr = "PIPELINING";
+ Policy policyValue = Policy.valueOf(policyStr);
+ jobSpec.setConnectorPolicyAssignmentPolicy(new HiveConnectorPolicyAssignmentPolicy(policyValue));
+ jobSpec.setUseConnectorPolicyForScheduling(false);
+ }
+
+ @Override
+ public int executeJob() {
+ try {
+ codeGen();
+ executeHyracksJob(jobSpec);
+ } catch (Exception e) {
+ e.printStackTrace();
+ return 1;
+ }
+ return 0;
+ }
+
+ private List<Operator> generateRootOperatorDAG(List<Task<? extends Serializable>> rootTasks,
+ HashMap<String, PartitionDesc> aliasToPath) {
+
+ List<Operator> rootOps = new ArrayList<Operator>();
+ List<Task<? extends Serializable>> toDelete = new ArrayList<Task<? extends Serializable>>();
+ tasksVisited.clear();
+
+ for (int i = rootTasks.size() - 1; i >= 0; i--) {
+ /**
+ * list of map-reduce tasks
+ */
+ Task<? extends Serializable> task = rootTasks.get(i);
+
+ if (task instanceof MapRedTask) {
+ List<Operator> mapRootOps = articulateMapReduceOperators(task, rootOps, aliasToPath, rootTasks);
+ if (i == 0)
+ rootOps.addAll(mapRootOps);
+ else {
+ List<Operator> leaves = new ArrayList<Operator>();
+ getLeaves(rootOps, leaves);
+
+ List<Operator> mapChildren = new ArrayList<Operator>();
+ for (Operator childMap : mapRootOps) {
+ if (childMap instanceof TableScanOperator) {
+ TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
+ if (topDesc == null)
+ mapChildren.add(childMap);
+ else {
+ rootOps.add(childMap);
+ }
+ } else
+ mapChildren.add(childMap);
+ }
+
+ if (mapChildren.size() > 0) {
+ for (Operator leaf : leaves)
+ leaf.setChildOperators(mapChildren);
+ for (Operator child : mapChildren)
+ child.setParentOperators(leaves);
+ }
+ }
+
+ MapredWork mr = (MapredWork) task.getWork();
+ HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
+
+ addAliasToPartition(aliasToPath, map);
+ toDelete.add(task);
+ }
+ }
+
+ for (Task<? extends Serializable> task : toDelete)
+ rootTasks.remove(task);
+
+ return rootOps;
+ }
+
+ private void addAliasToPartition(HashMap<String, PartitionDesc> aliasToPath, HashMap<String, PartitionDesc> map) {
+ Iterator<String> keys = map.keySet().iterator();
+ while (keys.hasNext()) {
+ String key = keys.next();
+ PartitionDesc part = map.get(key);
+ String[] names = key.split(":");
+ for (String name : names) {
+ aliasToPath.put(name, part);
+ }
+ }
+ }
+
+ private List<Operator> articulateMapReduceOperators(Task task, List<Operator> rootOps,
+ HashMap<String, PartitionDesc> aliasToPath, List<Task<? extends Serializable>> rootTasks) {
+ // System.out.println("!"+task.getName());
+ if (!(task instanceof MapRedTask)) {
+ if (!(task instanceof ConditionalTask)) {
+ rootTasks.add(task);
+ return null;
+ } else {
+ // remove map-reduce branches in condition task
+ ConditionalTask condition = (ConditionalTask) task;
+ List<Task<? extends Serializable>> branches = condition.getListTasks();
+ for (int i = branches.size() - 1; i >= 0; i--) {
+ Task branch = branches.get(i);
+ if (branch instanceof MapRedTask) {
+ return articulateMapReduceOperators(branch, rootOps, aliasToPath, rootTasks);
+ }
+ }
+ rootTasks.add(task);
+ return null;
+ }
+ }
+
+ MapredWork mr = (MapredWork) task.getWork();
+ HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
+
+ // put all aliasToParitionDesc mapping into the map
+ addAliasToPartition(aliasToPath, map);
+
+ MapRedTask mrtask = (MapRedTask) task;
+ MapredWork work = (MapredWork) mrtask.getWork();
+ HashMap<String, Operator<? extends Serializable>> operators = work.getAliasToWork();
+
+ Set entries = operators.entrySet();
+ Iterator<Entry<String, Operator>> iterator = entries.iterator();
+ List<Operator> mapRootOps = new ArrayList<Operator>();
+
+ // get map root operators
+ while (iterator.hasNext()) {
+ Operator next = iterator.next().getValue();
+ if (!mapRootOps.contains(next)) {
+ // clear that only for the case of union
+ mapRootOps.add(next);
+ }
+ }
+
+ // get map local work
+ MapredLocalWork localWork = work.getMapLocalWork();
+ if (localWork != null) {
+ HashMap<String, Operator<? extends Serializable>> localOperators = localWork.getAliasToWork();
+
+ Set localEntries = localOperators.entrySet();
+ Iterator<Entry<String, Operator>> localIterator = localEntries.iterator();
+ while (localIterator.hasNext()) {
+ mapRootOps.add(localIterator.next().getValue());
+ }
+
+ HashMap<String, FetchWork> localFetch = localWork.getAliasToFetchWork();
+ Set localFetchEntries = localFetch.entrySet();
+ Iterator<Entry<String, FetchWork>> localFetchIterator = localFetchEntries.iterator();
+ while (localFetchIterator.hasNext()) {
+ Entry<String, FetchWork> fetchMap = localFetchIterator.next();
+ FetchWork fetch = fetchMap.getValue();
+ String alias = fetchMap.getKey();
+ List<PartitionDesc> dirPart = fetch.getPartDesc();
+
+ // temporary hack: put the first partitionDesc into the map
+ aliasToPath.put(alias, dirPart.get(0));
+ }
+ }
+
+ Boolean visited = tasksVisited.get(task);
+ if (visited != null && visited.booleanValue() == true) {
+ return mapRootOps;
+ }
+
+ // do that only for union operator
+ for (Operator op : mapRootOps)
+ if (op.getParentOperators() != null)
+ op.getParentOperators().clear();
+
+ List<Operator> mapLeaves = new ArrayList<Operator>();
+ downToLeaves(mapRootOps, mapLeaves);
+ List<Operator> reduceOps = new ArrayList<Operator>();
+
+ if (work.getReducer() != null)
+ reduceOps.add(work.getReducer());
+
+ for (Operator mapLeaf : mapLeaves) {
+ mapLeaf.setChildOperators(reduceOps);
+ }
+
+ for (Operator reduceOp : reduceOps) {
+ if (reduceOp != null)
+ reduceOp.setParentOperators(mapLeaves);
+ }
+
+ List<Operator> leafs = new ArrayList<Operator>();
+ if (reduceOps.size() > 0) {
+ downToLeaves(reduceOps, leafs);
+ } else {
+ leafs = mapLeaves;
+ }
+
+ List<Operator> mapChildren = new ArrayList<Operator>();
+ if (task.getChildTasks() != null && task.getChildTasks().size() > 0) {
+ for (Object child : task.getChildTasks()) {
+ List<Operator> childMapOps = articulateMapReduceOperators((Task) child, rootOps, aliasToPath, rootTasks);
+ if (childMapOps == null)
+ continue;
+
+ for (Operator childMap : childMapOps) {
+ if (childMap instanceof TableScanOperator) {
+ TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
+ if (topDesc == null)
+ mapChildren.add(childMap);
+ else {
+ rootOps.add(childMap);
+ }
+ } else {
+ // if not table scan, add the child
+ mapChildren.add(childMap);
+ }
+ }
+ }
+
+ if (mapChildren.size() > 0) {
+ int i = 0;
+ for (Operator leaf : leafs) {
+ if (leaf.getChildOperators() == null || leaf.getChildOperators().size() == 0)
+ leaf.setChildOperators(new ArrayList<Operator>());
+ leaf.getChildOperators().add(mapChildren.get(i));
+ i++;
+ }
+ i = 0;
+ for (Operator child : mapChildren) {
+ if (child.getParentOperators() == null || child.getParentOperators().size() == 0)
+ child.setParentOperators(new ArrayList<Operator>());
+ child.getParentOperators().add(leafs.get(i));
+ i++;
+ }
+ }
+ }
+
+ // mark this task as visited
+ this.tasksVisited.put(task, true);
+ return mapRootOps;
+ }
+
+ /**
+ * down to leaf nodes
+ *
+ * @param ops
+ * @param leaves
+ */
+ private void downToLeaves(List<Operator> ops, List<Operator> leaves) {
+
+ // Operator currentOp;
+ for (Operator op : ops) {
+ if (op != null && op.getChildOperators() != null && op.getChildOperators().size() > 0) {
+ downToLeaves(op.getChildOperators(), leaves);
+ } else {
+ if (op != null && leaves.indexOf(op) < 0)
+ leaves.add(op);
+ }
+ }
+ }
+
+ private void getLeaves(List<Operator> roots, List<Operator> currentLeaves) {
+ for (Operator op : roots) {
+ List<Operator> children = op.getChildOperators();
+ if (children == null || children.size() <= 0) {
+ currentLeaves.add(op);
+ } else {
+ getLeaves(children, currentLeaves);
+ }
+ }
+ }
+
+ private void executeHyracksJob(JobSpecification job) throws Exception {
+
+ /**
+ * load the properties file if it is not loaded
+ */
+ if (clusterProps == null) {
+ clusterProps = new Properties();
+ InputStream confIn = new FileInputStream(clusterPropertiesPath);
+ clusterProps.load(confIn);
+ confIn.close();
+ }
+
+ if (hcc == null) {
+ BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));
+ String masterNode = ipReader.readLine();
+ ipReader.close();
+
+ InetAddress[] ips = InetAddress.getAllByName(masterNode);
+ int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));
+ for (InetAddress ip : ips) {
+ if (ip.getAddress().length <= 4) {
+ try {
+ hcc = new HyracksConnection(ip.getHostAddress(), port);
+ break;
+ } catch (Exception e) {
+ continue;
+ }
+ }
+ }
+ }
+
+ long start = System.currentTimeMillis();
+ JobId jobId = hcc.startJob(job);
+ hcc.waitForCompletion(jobId);
+
+ // System.out.println("job finished: " + jobId.toString());
+ // call all leave nodes to end
+ for (Operator leaf : leaveOps) {
+ jobClose(leaf);
+ }
+
+ long end = System.currentTimeMillis();
+ System.err.println(start + " " + end + " " + (end - start));
+ }
+
+ /**
+ * mv to final directory on hdfs (not real final)
+ *
+ * @param leaf
+ * @throws Exception
+ */
+ private void jobClose(Operator leaf) throws Exception {
+ FileSinkOperator fsOp = (FileSinkOperator) leaf;
+ FileSinkDesc desc = fsOp.getConf();
+ boolean isNativeTable = !desc.getTableInfo().isNonNative();
+ if ((conf != null) && isNativeTable) {
+ String specPath = desc.getDirName();
+ DynamicPartitionCtx dpCtx = desc.getDynPartCtx();
+ // for 0.7.0
+ fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);
+ // for 0.8.0
+ // Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx,
+ // desc);
+ }
+ }
+}
diff --git a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
index 4c40f5d..4ef74e9 100644
--- a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
@@ -444,10 +444,11 @@
// hyracks run
if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
- hivesterix = true;
- return engine.compileJob(sem.getRootTasks());
+ int engineRet = engine.compileJob(sem.getRootTasks());
+ if (engineRet == 0) {
+ hivesterix = true;
+ }
}
-
return 0;
} catch (SemanticException e) {
errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
diff --git a/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml b/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml
index 1fce28e..23a842a 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml
+++ b/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml
@@ -36,7 +36,8 @@
whereas hive uses -1 as its default value.
By setting this property to -1, Hive will automatically figure out what
should be the number of reducers.
- </description>
+ </description>
+ </property>
<property>
<name>hive.hyracks.connectorpolicy</name>
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
index d11b091..0e737c5 100755
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
@@ -47,10 +47,15 @@
IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
fi
else
- IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
- if [ "$IPADDR" = "" ]
+ #Get IP Address
+ IPADDR=`/sbin/ifconfig en0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ if [ "$IPADDR" = "" ]
then
- IPADDR=`/sbin/ifconfig lo0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ fi
+ if [ "$IPADDR" = "" ]
+ then
+ IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
fi
fi
diff --git a/hivesterix/hivesterix-runtime/pom.xml b/hivesterix/hivesterix-runtime/pom.xml
index 01db115..e4f5416 100644
--- a/hivesterix/hivesterix-runtime/pom.xml
+++ b/hivesterix/hivesterix-runtime/pom.xml
@@ -14,9 +14,7 @@
!-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hivesterix-runtime</artifactId>
- <version>0.2.7-SNAPSHOT</version>
<name>hivesterix-runtime</name>
<parent>
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
index 11a0558..3d70a40 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
@@ -15,12 +15,12 @@
package edu.uci.ics.hyracks.api.application;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
/**
* Application Context at the Node Controller for an application.
*
* @author vinayakb
- *
*/
public interface INCApplicationContext extends IApplicationContext {
/**
@@ -38,8 +38,7 @@
public IHyracksRootContext getRootContext();
/**
- * Set an object that can be later retrieved by the
- * {@link #getApplicationObject()} call.
+ * Set an object that can be later retrieved by the {@link #getApplicationObject()} call.
*
* @param object
* Application Object
@@ -47,10 +46,16 @@
public void setApplicationObject(Object object);
/**
- * Get the application object previously set by the
- * {@link #setApplicationObject(Object)} call.
+ * Get the application object previously set by the {@link #setApplicationObject(Object)} call.
*
* @return Application Object
*/
public Object getApplicationObject();
+
+ /**
+ * Get the memory manager at the node.
+ *
+ * @return Memory Manager
+ */
+ public IMemoryManager getMemoryManager();
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
index f5d7a90..c75f9f9 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
@@ -16,6 +16,7 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.IIOManager;
public interface IHyracksCommonContext {
@@ -23,5 +24,5 @@
public IIOManager getIOManager();
- public ByteBuffer allocateFrame();
+ public ByteBuffer allocateFrame() throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
index acdda02..0e3fefd 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
@@ -25,7 +25,8 @@
public int getInputArity();
- public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc);
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+ throws HyracksDataException;
public IFrameWriter getInputFrameWriter(int index);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/memory/IMemoryManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/memory/IMemoryManager.java
new file mode 100644
index 0000000..1017725
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/memory/IMemoryManager.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.api.resources.memory;
+
+public interface IMemoryManager {
+ public long getMaximumMemory();
+
+ public long getAvailableMemory();
+
+ public boolean allocate(long memory);
+
+ public void deallocate(long memory);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index 7d50fa0..ffed337 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -167,7 +167,8 @@
}
@Override
- public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc) {
+ public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc)
+ throws HyracksDataException {
/**
* set the right output frame writer
*/
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 3395d69..56b6654 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -20,6 +20,8 @@
import java.util.HashMap;
import java.util.Hashtable;
import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
@@ -56,6 +58,8 @@
import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
public class Joblet implements IHyracksJobletContext, ICounterContext {
+ private static final Logger LOGGER = Logger.getLogger(Joblet.class.getName());
+
private final NodeControllerService nodeController;
private final INCApplicationContext appCtx;
@@ -86,6 +90,8 @@
private final int frameSize;
+ private final AtomicLong memoryAllocation;
+
private JobStatus cleanupStatus;
private boolean cleanupPending;
@@ -97,6 +103,7 @@
this.deploymentId = deploymentId;
this.jobId = jobId;
this.frameSize = acg.getFrameSize();
+ memoryAllocation = new AtomicLong();
this.acg = acg;
partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
env = new OperatorEnvironmentImpl(nodeController.getId());
@@ -201,6 +208,11 @@
}
public void close() {
+ long stillAllocated = memoryAllocation.get();
+ if (stillAllocated > 0) {
+ LOGGER.warning("Freeing leaked " + stillAllocated + " bytes");
+ appCtx.getMemoryManager().deallocate(stillAllocated);
+ }
nodeController.getExecutor().execute(new Runnable() {
@Override
public void run() {
@@ -209,11 +221,20 @@
});
}
- ByteBuffer allocateFrame() {
- return ByteBuffer.allocate(getFrameSize());
+ ByteBuffer allocateFrame() throws HyracksDataException {
+ if (appCtx.getMemoryManager().allocate(frameSize)) {
+ memoryAllocation.addAndGet(frameSize);
+ return ByteBuffer.allocate(frameSize);
+ }
+ throw new HyracksDataException("Unable to allocate frame: Not enough memory");
}
- int getFrameSize() {
+ public void deallocateFrames(int nFrames) {
+ memoryAllocation.addAndGet(nFrames * frameSize);
+ appCtx.getMemoryManager().deallocate(nFrames * frameSize);
+ }
+
+ final int getFrameSize() {
return frameSize;
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 60859e3..6049a3b 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -69,6 +69,7 @@
import edu.uci.ics.hyracks.control.nc.net.DatasetNetworkManager;
import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
+import edu.uci.ics.hyracks.control.nc.resources.memory.MemoryManager;
import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
import edu.uci.ics.hyracks.control.nc.work.AbortTasksWork;
import edu.uci.ics.hyracks.control.nc.work.ApplicationMessageWork;
@@ -87,6 +88,8 @@
public class NodeControllerService extends AbstractRemoteService {
private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
+ private static final double MEMORY_FUDGE_FACTOR = 0.8;
+
private NCConfig ncConfig;
private final String id;
@@ -139,6 +142,8 @@
private final Mutable<FutureValue<Map<String, NodeControllerInfo>>> getNodeControllerInfosAcceptor;
+ private final MemoryManager memoryManager;
+
public NodeControllerService(NCConfig ncConfig) throws Exception {
this.ncConfig = ncConfig;
id = ncConfig.nodeId;
@@ -165,6 +170,7 @@
osMXBean = ManagementFactory.getOperatingSystemMXBean();
registrationPending = true;
getNodeControllerInfosAcceptor = new MutableObject<FutureValue<Map<String, NodeControllerInfo>>>();
+ memoryManager = new MemoryManager((long) (memoryMXBean.getHeapMemoryUsage().getMax() * MEMORY_FUDGE_FACTOR));
}
public IHyracksRootContext getRootContext() {
@@ -275,7 +281,7 @@
}
private void startApplication() throws Exception {
- appCtx = new NCApplicationContext(serverCtx, ctx, id);
+ appCtx = new NCApplicationContext(serverCtx, ctx, id, memoryManager);
String className = ncConfig.appNCMainClass;
if (className != null) {
Class<?> c = Class.forName(className);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index 1c92354..53e5a01 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -109,7 +109,7 @@
}
@Override
- public ByteBuffer allocateFrame() {
+ public ByteBuffer allocateFrame() throws HyracksDataException {
return joblet.allocateFrame();
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
index 8a1f0d9..4b8eb53 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
@@ -19,18 +19,23 @@
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.nc.resources.memory.MemoryManager;
public class NCApplicationContext extends ApplicationContext implements INCApplicationContext {
private final String nodeId;
private final IHyracksRootContext rootCtx;
+ private final MemoryManager memoryManager;
private Object appObject;
- public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String nodeId) throws IOException {
+ public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String nodeId,
+ MemoryManager memoryManager) throws IOException {
super(serverCtx);
this.nodeId = nodeId;
this.rootCtx = rootCtx;
+ this.memoryManager = memoryManager;
}
@Override
@@ -56,4 +61,9 @@
public Object getApplicationObject() {
return appObject;
}
+
+ @Override
+ public IMemoryManager getMemoryManager() {
+ return memoryManager;
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
new file mode 100644
index 0000000..8add260
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.control.nc.resources.memory;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
+
+public class MemoryManager implements IMemoryManager {
+ private final long maxMemory;
+
+ private final AtomicLong memory;
+
+ public MemoryManager(long maxMemory) {
+ this.maxMemory = maxMemory;
+ this.memory = new AtomicLong(maxMemory);
+ }
+
+ @Override
+ public long getMaximumMemory() {
+ return maxMemory;
+ }
+
+ @Override
+ public long getAvailableMemory() {
+ return memory.get();
+ }
+
+ @Override
+ public boolean allocate(long memory) {
+ if (this.memory.addAndGet(-memory) < 0) {
+ this.memory.addAndGet(memory);
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void deallocate(long memory) {
+ this.memory.addAndGet(memory);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index 424af4c..b94a219 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -34,7 +34,7 @@
private final FrameDeserializer frameDeserializer;
public FrameDeserializingDataReader(IHyracksTaskContext ctx, IFrameReader frameReader,
- RecordDescriptor recordDescriptor) {
+ RecordDescriptor recordDescriptor) throws HyracksDataException {
buffer = ctx.allocateFrame();
this.frameReader = frameReader;
this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 3d23c31..4392fd6 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -39,7 +39,8 @@
private boolean open;
- public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
+ public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter)
+ throws HyracksDataException {
buffer = ctx.allocateFrame();
tb = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
this.recordDescriptor = recordDescriptor;
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
index 5fbd2a5..f84ffda3 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
@@ -60,7 +60,7 @@
public ReduceWriter(IHyracksTaskContext ctx, HadoopHelper helper, int[] groupFields,
IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
Reducer<K2, V2, K3, V3> reducer, RecordWriter<K3, V3> recordWriter, TaskAttemptID taId,
- TaskAttemptContext taskAttemptContext) {
+ TaskAttemptContext taskAttemptContext) throws HyracksDataException {
this.ctx = ctx;
this.helper = helper;
this.groupFields = groupFields;
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index b34fa9f..30ba3ec 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -134,7 +134,7 @@
private RunFileWriter rfw;
private int blockId;
- public RunInfo() {
+ public RunInfo() throws HyracksDataException {
buffer = ctx.allocateFrame();
fta = new FrameTupleAppender(ctx.getFrameSize());
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
index 82a7797..d36b29e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -45,7 +45,7 @@
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
final FileSplit split = fileSplitProvider.getFileSplits()[partition];
final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
return new AbstractUnaryOutputSourceOperatorNodePushable() {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
index 7393072..e2518ee 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
@@ -17,7 +17,8 @@
import java.io.Serializable;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface ITupleParserFactory extends Serializable {
- public ITupleParser createTupleParser(IHyracksTaskContext ctx);
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 8cf929e..ecd5284 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -366,8 +366,9 @@
* When frames are already created, they are recycled.
*
* @return Whether a new frame is added successfully.
+ * @throws HyracksDataException
*/
- private boolean nextAvailableFrame() {
+ private boolean nextAvailableFrame() throws HyracksDataException {
// Return false if the number of frames is equal to the limit.
if (lastBufIndex + 1 >= framesLimit)
return false;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
index f11b12a..f8ba003 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -142,7 +142,7 @@
outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
}
- private void addNewBuffer() {
+ private void addNewBuffer() throws HyracksDataException {
ByteBuffer buffer = ctx.allocateFrame();
buffer.position(0);
buffer.limit(buffer.capacity());
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
index 11458ad..7773765 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -46,7 +46,7 @@
public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
- IFrameWriter writer) {
+ IFrameWriter writer) throws HyracksDataException {
this.groupFields = groupFields;
this.comparators = comparators;
this.aggregator = aggregator;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 37fa40b..1bfa413 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -181,7 +181,7 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) throws HyracksDataException {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(joinAid, 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -370,7 +370,8 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index 7585698..2f719fa 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -109,7 +109,7 @@
reloadFrame(outerBuffer);
}
- private void createAndCopyFrame(ByteBuffer outerBuffer) {
+ private void createAndCopyFrame(ByteBuffer outerBuffer) throws HyracksDataException {
ByteBuffer outerBufferCopy = ctx.allocateFrame();
FrameUtils.copy(outerBuffer, outerBufferCopy);
outBuffers.add(outerBufferCopy);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 47a9a1c..cd32c81 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -39,8 +39,8 @@
/**
* @author pouria
- This class mainly applies one level of HHJ on a pair of
- relations. It is always called by the descriptor.
+ * This class mainly applies one level of HHJ on a pair of
+ * relations. It is always called by the descriptor.
*/
public class OptimizedHybridHashJoin {
@@ -49,7 +49,7 @@
private final int INVALID_BUFFER = -2;
private final int UNALLOCATED_FRAME = -3;
private final int BUFFER_FOR_RESIDENT_PARTS = -1;
-
+
private IHyracksTaskContext ctx;
private final String rel0Name;
@@ -107,10 +107,10 @@
this.memForJoin = memForJoin;
this.buildRd = buildRd;
this.probeRd = probeRd;
- this.buildHpc = buildHpc;
- this.probeHpc = probeHpc;
- this.buildKeys = keys1;
- this.probeKeys = keys0;
+ this.buildHpc = buildHpc;
+ this.probeHpc = probeHpc;
+ this.buildKeys = keys1;
+ this.probeKeys = keys0;
this.comparators = comparators;
this.rel0Name = rel0Name;
this.rel1Name = rel1Name;
@@ -136,10 +136,10 @@
this.memForJoin = memForJoin;
this.buildRd = buildRd;
this.probeRd = probeRd;
- this.buildHpc = buildHpc;
- this.probeHpc = probeHpc;
- this.buildKeys = keys1;
- this.probeKeys = keys0;
+ this.buildHpc = buildHpc;
+ this.probeHpc = probeHpc;
+ this.buildKeys = keys1;
+ this.probeKeys = keys0;
this.comparators = comparators;
this.rel0Name = rel0Name;
this.rel1Name = rel1Name;
@@ -162,7 +162,7 @@
}
}
- public void initBuild() {
+ public void initBuild() throws HyracksDataException {
memBuffs = new ByteBuffer[memForJoin];
curPBuff = new int[numOfPartitions];
nextBuff = new int[memForJoin];
@@ -191,12 +191,12 @@
public void build(ByteBuffer buffer) throws HyracksDataException {
accessorBuild.reset(buffer);
int tupleCount = accessorBuild.getTupleCount();
-
+
boolean print = false;
- if(print){
- accessorBuild.prettyPrint();
- }
-
+ if (print) {
+ accessorBuild.prettyPrint();
+ }
+
for (int i = 0; i < tupleCount; ++i) {
int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
processTuple(i, pid);
@@ -225,7 +225,7 @@
buildTupAppender.reset(memBuffs[pidToSpill], true);
processTuple(tid, pid);
break;
- } //New Buffer allocated successfully
+ } //New Buffer allocated successfully
partition = memBuffs[curPBuff[pid]]; //Current Buffer for the partition is now updated by allocateFreeBuffer() call above
buildTupAppender.reset(partition, true);
if (!buildTupAppender.append(accessorBuild, tid)) {
@@ -250,7 +250,7 @@
}
}
- private int allocateFreeBuffer(int pid) {
+ private int allocateFreeBuffer(int pid) throws HyracksDataException {
if (nextFreeBuffIx != NO_MORE_FREE_BUFFER) {
if (memBuffs[nextFreeBuffIx] == null) {
memBuffs[nextFreeBuffIx] = ctx.allocateFrame();
@@ -425,7 +425,7 @@
private ArrayList<Integer> selectPartitionsToReload() {
ArrayList<Integer> p = new ArrayList<Integer>();
for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) {
- if (buildPSizeInFrames[i]>0 && (freeFramesCounter - buildPSizeInFrames[i] >= 0) ) {
+ if (buildPSizeInFrames[i] > 0 && (freeFramesCounter - buildPSizeInFrames[i] >= 0)) {
p.add(i);
freeFramesCounter -= buildPSizeInFrames[i];
}
@@ -457,7 +457,7 @@
}
}
- public void initProbe() {
+ public void initProbe() throws HyracksDataException {
sPartBuffs = new ByteBuffer[numOfSpilledParts];
for (int i = 0; i < numOfSpilledParts; i++) {
@@ -468,7 +468,7 @@
/* We only need to allocate one frame per spilled partition.
* Resident partitions do not need frames in probe, as their tuples join
* immediately with the resident build tuples using the inMemoryHashJoin */
- for (int i = 0; i < numOfPartitions; i++) {
+ for (int i = 0; i < numOfPartitions; i++) {
curPBuff[i] = (pStatus.get(i)) ? nextBuffIxToAlloc++ : BUFFER_FOR_RESIDENT_PARTS;
}
probePSizeInTups = new int[numOfPartitions];
@@ -488,10 +488,10 @@
int tupleCount = accessorProbe.getTupleCount();
boolean print = false;
- if(print){
- accessorProbe.prettyPrint();
- }
-
+ if (print) {
+ accessorProbe.prettyPrint();
+ }
+
if (numOfSpilledParts == 0) {
inMemJoiner.join(buffer, writer);
return;
@@ -508,14 +508,14 @@
probeTupAppenderToSpilled.reset(buff, needToClear);
if (probeTupAppenderToSpilled.append(accessorProbe, i)) {
break;
- }
+ }
probeWrite(pid, buff);
buff.clear();
needToClear = true;
}
} else { //pid is Resident
while (true) {
- if (probeTupAppenderToResident.append(accessorProbe, i)){
+ if (probeTupAppenderToResident.append(accessorProbe, i)) {
break;
}
inMemJoiner.join(probeResBuff, writer);
@@ -635,8 +635,8 @@
+ freeFramesCounter;
return s;
}
-
- public boolean isTableEmpty(){
- return this.isTableEmpty;
+
+ public boolean isTableEmpty() {
+ return this.isTableEmpty;
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 35d2468..95b7a3c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -335,7 +335,8 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
index 7330590..42ed59e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -57,7 +57,8 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
final ByteBuffer outputBuffer = ctx.allocateFrame();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
index d1d0d5c..110bddb 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
@@ -632,7 +632,7 @@
BSTNodeUtil.setPrev(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
}
- private ByteBuffer allocateFrame() {
+ private ByteBuffer allocateFrame() throws HyracksDataException {
return ctx.allocateFrame();
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 6e297a7..b149e30 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -36,7 +36,7 @@
public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDesc, int framesLimit) {
+ RecordDescriptor recordDesc, int framesLimit) throws HyracksDataException {
this.ctx = ctx;
frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
runs = new LinkedList<IFrameReader>();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index cb7b311..db70a62 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -50,7 +50,7 @@
public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDescriptor) {
+ RecordDescriptor recordDescriptor) throws HyracksDataException {
this.ctx = ctx;
this.sortFields = sortFields;
nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
@@ -76,7 +76,7 @@
return dataFrameCount;
}
- public void insertFrame(ByteBuffer buffer) {
+ public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
ByteBuffer copyFrame;
if (dataFrameCount == buffers.size()) {
copyFrame = ctx.allocateFrame();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
index 7cbf41b..ce6c00c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
@@ -14,9 +14,11 @@
*/
package edu.uci.ics.hyracks.dataflow.std.structures;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
public interface ISerializableTable {
- public void insert(int entry, TuplePointer tuplePointer);
+ public void insert(int entry, TuplePointer tuplePointer) throws HyracksDataException;
public void getTuplePointer(int entry, int offset, TuplePointer tuplePointer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
index 54dd7a7..a245c9d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -18,6 +18,7 @@
import java.util.List;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
/**
* An entry in the table is: #elements, #no-empty elements; fIndex, tIndex;
@@ -38,7 +39,7 @@
private int headerFrameCount = 0;
private TuplePointer tempTuplePointer = new TuplePointer();
- public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) {
+ public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) throws HyracksDataException {
this.ctx = ctx;
int frameSize = ctx.getFrameSize();
@@ -53,7 +54,7 @@
}
@Override
- public void insert(int entry, TuplePointer pointer) {
+ public void insert(int entry, TuplePointer pointer) throws HyracksDataException {
int hFrameIndex = getHeaderFrameIndex(entry);
int headerOffset = getHeaderFrameOffset(entry);
IntSerDeBuffer header = headers[hFrameIndex];
@@ -143,7 +144,8 @@
currentLargestFrameIndex = 0;
}
- private void insertNewEntry(IntSerDeBuffer header, int headerOffset, int entryCapacity, TuplePointer pointer) {
+ private void insertNewEntry(IntSerDeBuffer header, int headerOffset, int entryCapacity, TuplePointer pointer)
+ throws HyracksDataException {
IntSerDeBuffer lastFrame = contents.get(currentLargestFrameIndex);
int lastIndex = frameCurrentIndex.get(currentLargestFrameIndex);
int requiredIntCapacity = entryCapacity * 2;
@@ -191,7 +193,7 @@
}
private void insertNonFirstTuple(IntSerDeBuffer header, int headerOffset, int frameIndex, int offsetIndex,
- TuplePointer pointer) {
+ TuplePointer pointer) throws HyracksDataException {
IntSerDeBuffer frame = contents.get(frameIndex);
int entryItems = frame.getInt(offsetIndex);
int entryUsedItems = frame.getInt(offsetIndex + 1);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index c89bfba..aff4273 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -40,7 +40,8 @@
}
@Override
- public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+ throws HyracksDataException {
delegate.setDataWriter(index, new SerializingDataWriter(ctx, recordDesc, writer));
}
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 1431d0f..a027570 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -149,7 +149,8 @@
IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, btreeSplitProvider, typeTraits, comparatorFactories, null,
- fieldPermutation, 0.7f, false, 1000L, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+ fieldPermutation, 0.7f, false, 1000L, true, dataflowHelperFactory,
+ NoOpOperationCallbackFactory.INSTANCE);
JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
// distribute the records from the datagen via hashing to the bulk load
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index b484f85..805660f 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -141,7 +141,8 @@
IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.secondaryBTreeName);
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, btreeSplitProvider, secondaryTypeTraits, comparatorFactories, null,
- fieldPermutation, 0.7f, false, 1000L, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+ fieldPermutation, 0.7f, false, 1000L, true, dataflowHelperFactory,
+ NoOpOperationCallbackFactory.INSTANCE);
JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
// connect the ops
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index 4ac68e7..ed4cf0c 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -59,7 +59,7 @@
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
final ByteBuffer outputFrame = ctx.allocateFrame();
final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index a93c199..eec7225 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -178,7 +178,7 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- primaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, dataflowHelperFactory,
+ primaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true, dataflowHelperFactory,
NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
@@ -241,7 +241,7 @@
int[] fieldPermutation = { 3, 0 };
TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L,
+ secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true,
dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
index 8bb698a..1bd22d7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
@@ -203,8 +203,8 @@
int[] fieldPermutation = { 0, 1 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, primaryFileSplitProvider, primaryTypeTraits,
- primaryComparatorFactories, null, fieldPermutation, 0.7f, true, 1000L, btreeDataflowHelperFactory,
- NoOpOperationCallbackFactory.INSTANCE);
+ primaryComparatorFactories, null, fieldPermutation, 0.7f, true, 1000L, true,
+ btreeDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
return primaryBtreeBulkLoad;
}
@@ -277,7 +277,7 @@
private IOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec, int[] fieldPermutation) {
LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
- spec, fieldPermutation, true, 1000L, storageManager, btreeFileSplitProvider, lcManagerProvider,
+ spec, fieldPermutation, true, 1000L, true, storageManager, btreeFileSplitProvider, lcManagerProvider,
tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
tokenizerFactory, invertedIndexDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexBulkLoadOp, NC1_ID);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 00296a5..d1c682c 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -223,7 +223,7 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
TreeIndexBulkLoadOperatorDescriptor primaryBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- null, fieldPermutation, 0.7f, false, 1000L, btreeDataflowHelperFactory,
+ null, fieldPermutation, 0.7f, false, 1000L, true, btreeDataflowHelperFactory,
NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBulkLoad, NC1_ID);
@@ -280,8 +280,8 @@
int[] fieldPermutation = { 6, 7, 8, 9, 0 };
TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, null, fieldPermutation, 0.7f, false, 1000L, rtreeDataflowHelperFactory,
- NoOpOperationCallbackFactory.INSTANCE);
+ secondaryComparatorFactories, null, fieldPermutation, 0.7f, false, 1000L, true,
+ rtreeDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primarySearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index c6efca4..29be04b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -56,7 +56,7 @@
buffers = new ArrayList<ByteBuffer>();
}
- public IOpenableDataWriter<Object[]> createWriter() {
+ public IOpenableDataWriter<Object[]> createWriter() throws HyracksDataException {
return new SerializingDataWriter(ctx, rDes, new IFrameWriter() {
@Override
public void open() throws HyracksDataException {
@@ -79,7 +79,7 @@
});
}
- public IOpenableDataReader<Object[]> createDataReader() {
+ public IOpenableDataReader<Object[]> createDataReader() throws HyracksDataException {
return new FrameDeserializingDataReader(ctx, new IFrameReader() {
private int i;
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
index 71a7680..fbac95b 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -33,7 +33,8 @@
private static final long serialVersionUID = 1L;
@Override
- public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx) {
+ public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx)
+ throws HyracksDataException {
final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
final ByteBuffer buffer = ctx.allocateFrame();
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index 0e796b0..6811e1b 100644
--- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -209,15 +209,15 @@
throw new HyracksDataException("Failed to create the bloom filter builder since it is not activated.");
}
- this.numElements = numElements;
+ this.numElements = numElements == 0 ? 1 : numElements;
this.numHashes = numHashes;
- numBits = numElements * numBitsPerElement;
+ numBits = this.numElements * numBitsPerElement;
long tmp = (long) Math.ceil(numBits / (double) numBitsPerPage);
if (tmp > Integer.MAX_VALUE) {
throw new HyracksDataException("Cannot create a bloom filter with his huge number of pages.");
}
numPages = (int) tmp;
- if (numElements > 0) {
+ if (this.numElements > 0) {
persistBloomFilterMetaData();
readBloomFilterMetaData();
int currentPageId = 1;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 7c29fc9..689843b 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -934,8 +934,8 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws TreeIndexException {
try {
return new BTreeBulkLoader(fillFactor, verifyInput);
} catch (HyracksDataException e) {
@@ -1025,7 +1025,8 @@
HyracksDataException {
// New tuple should be strictly greater than last tuple.
if (cmp.compare(tuple, prevTuple) <= 0) {
- throw new UnsortedInputException("Input stream given to BTree bulk load is not sorted.");
+ throw new UnsortedInputException(
+ "Input stream given to BTree bulk load is not sorted or has duplicates.");
}
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
index 074ae7c..4006cb7 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
@@ -91,9 +91,10 @@
* the callback to be used for modification operations
* @param searchCallback
* the callback to be used for search operations
+ * @throws HyracksDataException
*/
public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback);
+ ISearchOperationCallback searchCallback) throws HyracksDataException;
/**
* Ensures that all pages (and tuples) of the index are logically consistent.
@@ -119,6 +120,6 @@
* @param verifyInput
* @throws IndexException
*/
- public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws IndexException;
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws IndexException;
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
index 31fc627..974d91c 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
@@ -14,11 +14,12 @@
*/
package edu.uci.ics.hyracks.storage.am.common.api;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
public interface IIndexOperationContext {
- void setOperation(IndexOperation newOp);
-
+ void setOperation(IndexOperation newOp) throws HyracksDataException;
+
IndexOperation getOperation();
void reset();
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
index 0504a94..a0f9d8e 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
@@ -34,6 +34,7 @@
private final float fillFactor;
private final boolean verifyInput;
private final long numElementsHint;
+ private final boolean checkIfEmptyIndex;
private final IIndexDataflowHelper indexHelper;
private FrameTupleAccessor accessor;
private IIndex index;
@@ -43,13 +44,14 @@
public IndexBulkLoadOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
int[] fieldPermutation, float fillFactor, boolean verifyInput, long numElementsHint,
- IRecordDescriptorProvider recordDescProvider) {
+ boolean checkIfEmptyIndex, IRecordDescriptorProvider recordDescProvider) {
this.opDesc = opDesc;
this.ctx = ctx;
this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
this.fillFactor = fillFactor;
this.verifyInput = verifyInput;
this.numElementsHint = numElementsHint;
+ this.checkIfEmptyIndex = checkIfEmptyIndex;
this.recDescProvider = recordDescProvider;
tuple.setFieldPermutation(fieldPermutation);
}
@@ -61,7 +63,7 @@
indexHelper.open();
index = indexHelper.getIndexInstance();
try {
- bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint);
+ bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
} catch (Exception e) {
indexHelper.close();
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
index 8c2d847..5e07e0c 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -75,7 +75,7 @@
// any physical artifact that the LocalResourceRepository is managing (e.g. a file containing the resource ID).
// Once the index has been created, a new resource ID can be generated.
if (resourceID != -1) {
- localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
+ localResourceRepository.deleteResourceByName(file.getFile().getPath());
}
index.create();
try {
@@ -83,9 +83,8 @@
resourceID = resourceIdFactory.createId();
ILocalResourceFactory localResourceFactory = opDesc.getLocalResourceFactoryProvider()
.getLocalResourceFactory();
- localResourceRepository.insert(
- localResourceFactory.createLocalResource(resourceID, file.getFile().getPath(), partition),
- ioDeviceId);
+ localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+ .getPath(), partition));
} catch (IOException e) {
throw new HyracksDataException(e);
}
@@ -127,7 +126,7 @@
}
if (resourceID != -1) {
- localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
+ localResourceRepository.deleteResourceByName(file.getFile().getPath());
}
index.destroy();
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index 09cdd68..a58e9e8 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -36,12 +36,13 @@
private final float fillFactor;
private final boolean verifyInput;
private final long numElementsHint;
+ private final boolean checkIfEmptyIndex;
public TreeIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec,
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
- float fillFactor, boolean verifyInput, long numElementsHint,
+ float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
IIndexDataflowHelperFactory dataflowHelperFactory,
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 1, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
@@ -52,12 +53,13 @@
this.fillFactor = fillFactor;
this.verifyInput = verifyInput;
this.numElementsHint = numElementsHint;
+ this.checkIfEmptyIndex = checkIfEmptyIndex;
}
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new IndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor, verifyInput,
- numElementsHint, recordDescProvider);
+ numElementsHint, checkIfEmptyIndex, recordDescProvider);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index b573a32..19d40a0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -333,8 +333,12 @@
}
public class TreeIndexInsertBulkLoader implements IIndexBulkLoader {
- ITreeIndexAccessor accessor = (ITreeIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
+ ITreeIndexAccessor accessor;
+
+ public TreeIndexInsertBulkLoader() throws HyracksDataException {
+ accessor = (ITreeIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ }
@Override
public void add(ITupleReference tuple) throws HyracksDataException {
@@ -356,4 +360,4 @@
public long getMemoryAllocationSize() {
return 0;
}
-}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index d34906e..f217419 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -49,11 +49,10 @@
@Override
public ITreeIndex createIndexInstance() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
- return LSMBTreeUtils.createLSMTree(virtualBufferCache, ctx.getIOManager(), file, opDesc.getStorageManager()
- .getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc
- .getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
- .getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory
- .getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider, opDesc.getFileSplitProvider()
- .getFileSplits()[partition].getIODeviceId());
+ return LSMBTreeUtils.createLSMTree(virtualBufferCache, file, opDesc.getStorageManager().getBufferCache(ctx),
+ opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
+ treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
+ bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+ ioOpCallbackProvider);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 8161200..3557d73 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -211,6 +211,7 @@
List<ILSMComponent> immutableComponents = componentsRef.get();
mutableComponent.getBTree().clear();
+ mutableComponent.reset();
for (ILSMComponent c : immutableComponents) {
LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
component.getBloomFilter().deactivate();
@@ -369,7 +370,7 @@
LSMBTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
flushOp.getBloomFilterFlushTarget(), true);
- IIndexBulkLoader bulkLoader = component.getBTree().createBulkLoader(1.0f, false, numElements);
+ IIndexBulkLoader bulkLoader = component.getBTree().createBulkLoader(1.0f, false, numElements, false);
IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numElements,
bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
@@ -429,7 +430,7 @@
LSMBTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
- IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, numElements);
+ IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, numElements, false);
IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements,
bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
try {
@@ -464,10 +465,10 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws TreeIndexException {
try {
- return new LSMBTreeBulkLoader(fillLevel, verifyInput, numElementsHint);
+ return new LSMBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex);
} catch (HyracksDataException e) {
throw new TreeIndexException(e);
}
@@ -498,19 +499,22 @@
private final ILSMComponent component;
private final BTreeBulkLoader bulkLoader;
private final IIndexBulkLoader builder;
- private boolean endHasBeenCalled = false;
+ private boolean cleanedUpArtifacts = false;
+ private boolean isEmptyComponent = true;
+ private boolean endedBloomFilterLoad = false;
- public LSMBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+ public LSMBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex)
throws TreeIndexException, HyracksDataException {
+ if (checkIfEmptyIndex && !isEmptyIndex()) {
+ throw new TreeIndexException("Cannot load an index that is not empty");
+ }
try {
component = createBulkLoadTarget();
- } catch (HyracksDataException e) {
- throw new TreeIndexException(e);
- } catch (IndexException e) {
+ } catch (HyracksDataException | IndexException e) {
throw new TreeIndexException(e);
}
bulkLoader = (BTreeBulkLoader) ((LSMBTreeImmutableComponent) component).getBTree().createBulkLoader(
- fillFactor, verifyInput, numElementsHint);
+ fillFactor, verifyInput, numElementsHint, false);
int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
@@ -524,36 +528,45 @@
try {
bulkLoader.add(tuple);
builder.add(tuple);
- } catch (IndexException e) {
- handleException();
+ } catch (IndexException | HyracksDataException | RuntimeException e) {
+ cleanupArtifacts();
throw e;
- } catch (HyracksDataException e) {
- handleException();
- throw e;
- } catch (RuntimeException e) {
- handleException();
- throw e;
+ }
+ if (isEmptyComponent) {
+ isEmptyComponent = false;
}
}
- protected void handleException() throws HyracksDataException, IndexException {
- if (!endHasBeenCalled) {
- builder.end();
+ protected void cleanupArtifacts() throws HyracksDataException, IndexException {
+ if (!cleanedUpArtifacts) {
+ cleanedUpArtifacts = true;
+ // We make sure to end the bloom filter load to release latches.
+ if (!endedBloomFilterLoad) {
+ builder.end();
+ endedBloomFilterLoad = true;
+ }
+ ((LSMBTreeImmutableComponent) component).getBTree().deactivate();
+ ((LSMBTreeImmutableComponent) component).getBTree().destroy();
+ ((LSMBTreeImmutableComponent) component).getBloomFilter().deactivate();
+ ((LSMBTreeImmutableComponent) component).getBloomFilter().destroy();
}
- ((LSMBTreeImmutableComponent) component).getBTree().deactivate();
- ((LSMBTreeImmutableComponent) component).getBTree().destroy();
- ((LSMBTreeImmutableComponent) component).getBloomFilter().deactivate();
- ((LSMBTreeImmutableComponent) component).getBloomFilter().destroy();
}
@Override
public void end() throws HyracksDataException, IndexException {
- bulkLoader.end();
- builder.end();
- endHasBeenCalled = true;
- lsmHarness.addBulkLoadedComponent(component);
+ if (!cleanedUpArtifacts) {
+ if (!endedBloomFilterLoad) {
+ builder.end();
+ endedBloomFilterLoad = true;
+ }
+ bulkLoader.end();
+ if (isEmptyComponent) {
+ cleanupArtifacts();
+ } else {
+ lsmHarness.addBulkLoadedComponent(component);
+ }
+ }
}
-
}
public LSMBTreeOpContext createOpContext(IModificationOperationCallback modificationCallback,
@@ -630,10 +643,7 @@
}
public boolean isEmptyIndex() throws HyracksDataException {
- List<ILSMComponent> immutableComponents = componentsRef.get();
- return immutableComponents.isEmpty()
- && mutableComponent.getBTree().isEmptyTree(
- mutableComponent.getBTree().getInteriorFrameFactory().createFrame());
+ return componentsRef.get().isEmpty() && !mutableComponent.isModified();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
index b71e9be..7e8fbb3 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -26,7 +26,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -39,9 +38,9 @@
private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
- public LSMBTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> btreeFactory, int ioDeviceId) {
- super(ioManager, fileMapProvider, file, null, ioDeviceId);
+ public LSMBTreeFileManager(IFileMapProvider fileMapProvider, FileReference file,
+ TreeIndexFactory<? extends ITreeIndex> btreeFactory) {
+ super(fileMapProvider, file, null);
this.btreeFactory = btreeFactory;
}
@@ -79,16 +78,16 @@
ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
- // Gather files from the IODeviceHandle.
+ // Gather files
// List of valid BTree files.
- cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+ cleanupAndGetValidFilesInternal(btreeFilter, btreeFactory, allBTreeFiles);
HashSet<String> btreeFilesSet = new HashSet<String>();
for (ComparableFileName cmpFileName : allBTreeFiles) {
int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
}
- validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+ validateFiles(btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
// Sanity check.
if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 6a94150..ac62d9f 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -52,13 +52,12 @@
ISearchOperationCallback searchCallback, int numBloomFilterKeyFields) {
IBinaryComparatorFactory cmpFactories[] = memBTree.getComparatorFactories();
if (cmpFactories[0] != null) {
- this.cmp = MultiComparator.createIgnoreFieldLength(memBTree.getComparatorFactories());
+ this.cmp = MultiComparator.create(memBTree.getComparatorFactories());
} else {
this.cmp = null;
}
- bloomFilterCmp = MultiComparator.createIgnoreFieldLength(memBTree.getComparatorFactories(), 0,
- numBloomFilterKeyFields);
+ bloomFilterCmp = MultiComparator.create(memBTree.getComparatorFactories(), 0, numBloomFilterKeyFields);
this.memBTree = memBTree;
this.insertLeafFrameFactory = insertLeafFrameFactory;
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
index 28bcb45..507ec02 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -18,7 +18,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
@@ -44,22 +43,11 @@
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class LSMBTreeUtils {
-
- public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
- return createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
- cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackProvider, 0);
- }
-
- public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) {
+ public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
cmpFactories.length, false);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -83,8 +71,7 @@
BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
bloomFilterKeyFields);
- ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
- diskBTreeFactory, ioDeviceId);
+ ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(diskFileMapProvider, file, diskBTreeFactory);
LSMBTree lsmTree = new LSMBTree(virtualBufferCache, interiorFrameFactory, insertLeafFrameFactory,
deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index 3f71c67..a6425ce 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -28,8 +28,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
@@ -45,7 +43,6 @@
protected static final String BLOOM_FILTER_STRING = "f";
protected final IFileMapProvider fileMapProvider;
- protected final IODeviceHandle dev;
// baseDir should reflect dataset name and partition name.
protected String baseDir;
@@ -55,15 +52,14 @@
protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
- public AbstractLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
+ public AbstractLSMIndexFileManager(IFileMapProvider fileMapProvider, FileReference file,
+ TreeIndexFactory<? extends ITreeIndex> treeFactory) {
this.baseDir = file.getFile().getPath();
if (!baseDir.endsWith(System.getProperty("file.separator"))) {
baseDir += System.getProperty("file.separator");
}
this.fileMapProvider = fileMapProvider;
this.treeFactory = treeFactory;
- this.dev = ioManager.getIODevices().get(ioDeviceId);
}
private static FilenameFilter fileNameFilter = new FilenameFilter() {
@@ -94,10 +90,10 @@
}
}
- protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
+ protected void cleanupAndGetValidFilesInternal(FilenameFilter filter,
TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
throws HyracksDataException, IndexException {
- File dir = new File(dev.getPath(), baseDir);
+ File dir = new File(baseDir);
String[] files = dir.list(filter);
for (String fileName : files) {
File file = new File(dir.getPath() + File.separator + fileName);
@@ -110,11 +106,11 @@
}
}
- protected void validateFiles(IODeviceHandle dev, HashSet<String> groundTruth,
- ArrayList<ComparableFileName> validFiles, FilenameFilter filter,
- TreeIndexFactory<? extends ITreeIndex> treeFactory) throws HyracksDataException, IndexException {
+ protected void validateFiles(HashSet<String> groundTruth, ArrayList<ComparableFileName> validFiles,
+ FilenameFilter filter, TreeIndexFactory<? extends ITreeIndex> treeFactory) throws HyracksDataException,
+ IndexException {
ArrayList<ComparableFileName> tmpAllInvListsFiles = new ArrayList<ComparableFileName>();
- cleanupAndGetValidFilesInternal(dev, filter, treeFactory, tmpAllInvListsFiles);
+ cleanupAndGetValidFilesInternal(filter, treeFactory, tmpAllInvListsFiles);
for (ComparableFileName cmpFileName : tmpAllInvListsFiles) {
int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
String file = cmpFileName.fileName.substring(0, index);
@@ -129,13 +125,13 @@
@Override
public void createDirs() {
- File f = new File(dev.getPath(), baseDir);
+ File f = new File(baseDir);
f.mkdirs();
}
@Override
public void deleteDirs() {
- File f = new File(dev.getPath(), baseDir);
+ File f = new File(baseDir);
delete(f);
}
@@ -155,7 +151,7 @@
};
protected FileReference createFlushFile(String relFlushFileName) {
- return dev.createFileReference(relFlushFileName);
+ return new FileReference(new File(relFlushFileName));
}
protected FileReference createMergeFile(String relMergeFileName) {
@@ -185,12 +181,12 @@
List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
ArrayList<ComparableFileName> allFiles = new ArrayList<ComparableFileName>();
- // Gather files from the IODeviceHandle and delete invalid files
+ // Gather files and delete invalid files
// There are two types of invalid files:
// (1) The isValid flag is not set
// (2) The file's interval is contained by some other file
// Here, we only filter out (1).
- cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
+ cleanupAndGetValidFilesInternal(fileNameFilter, treeFactory, allFiles);
if (allFiles.isEmpty()) {
return validFiles;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
index 425af8d..1c4235c 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
@@ -38,13 +38,15 @@
private final int[] fieldPermutation;
private final boolean verifyInput;
private final long numElementsHint;
+ private final boolean checkIfEmptyIndex;
public LSMInvertedIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] fieldPermutation,
- boolean verifyInput, long numElementsHint, IStorageManagerInterface storageManager,
- IFileSplitProvider fileSplitProvider, IIndexLifecycleManagerProvider lifecycleManagerProvider,
- ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
- ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
- IBinaryTokenizerFactory tokenizerFactory, IIndexDataflowHelperFactory invertedIndexDataflowHelperFactory,
+ boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
+ IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+ IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+ IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
+ IIndexDataflowHelperFactory invertedIndexDataflowHelperFactory,
IModificationOperationCallbackFactory modificationOpCallbackFactory) {
super(spec, 1, 0, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
@@ -53,12 +55,13 @@
this.fieldPermutation = fieldPermutation;
this.verifyInput = verifyInput;
this.numElementsHint = numElementsHint;
+ this.checkIfEmptyIndex = checkIfEmptyIndex;
}
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new IndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, 1.0f, verifyInput,
- numElementsHint, recordDescProvider);
+ numElementsHint, checkIfEmptyIndex, recordDescProvider);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index 7ec7c84..1afd9d5 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -59,9 +59,8 @@
diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
- diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
- mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider,
- opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+ diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+ opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
return invIndex;
} catch (IndexException e) {
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index 4a4956f..0dbd06b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -59,9 +59,8 @@
virtualBufferCache, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
- diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
- mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider,
- opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+ diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+ opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
return invIndex;
} catch (IndexException e) {
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 20d3e7f..3eae5a7 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -43,7 +43,6 @@
import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -175,6 +174,7 @@
List<ILSMComponent> immutableComponents = componentsRef.get();
mutableComponent.getInvIndex().clear();
mutableComponent.getDeletedKeysBTree().clear();
+ mutableComponent.reset();
for (ILSMComponent c : immutableComponents) {
LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
component.getBloomFilter().deactivate();
@@ -432,7 +432,7 @@
memBTreeAccessor.search(scanCursor, nullPred);
// Bulk load the disk inverted index from the in-memory inverted index.
- IIndexBulkLoader invIndexBulkLoader = diskInvertedIndex.createBulkLoader(1.0f, false, 0L);
+ IIndexBulkLoader invIndexBulkLoader = diskInvertedIndex.createBulkLoader(1.0f, false, 0L, false);
try {
while (scanCursor.hasNext()) {
scanCursor.next();
@@ -472,7 +472,7 @@
deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred);
// Bulk load the deleted-keys BTree.
- IIndexBulkLoader deletedKeysBTreeBulkLoader = diskDeletedKeysBTree.createBulkLoader(1.0f, false, 0L);
+ IIndexBulkLoader deletedKeysBTreeBulkLoader = diskDeletedKeysBTree.createBulkLoader(1.0f, false, 0L, false);
IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numBTreeTuples,
bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
@@ -534,7 +534,7 @@
IInvertedIndex mergedDiskInvertedIndex = component.getInvIndex();
IIndexCursor cursor = mergeOp.getCursor();
- IIndexBulkLoader invIndexBulkLoader = mergedDiskInvertedIndex.createBulkLoader(1.0f, true, 0L);
+ IIndexBulkLoader invIndexBulkLoader = mergedDiskInvertedIndex.createBulkLoader(1.0f, true, 0L, false);
try {
while (cursor.hasNext()) {
cursor.next();
@@ -559,63 +559,76 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws IndexException {
- return new LSMInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint);
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws IndexException {
+ try {
+ return new LSMInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ } catch (HyracksDataException e) {
+ throw new IndexException(e);
+ }
+ }
+
+ public boolean isEmptyIndex() throws HyracksDataException {
+ return componentsRef.get().isEmpty() && !mutableComponent.isModified();
}
public class LSMInvertedIndexBulkLoader implements IIndexBulkLoader {
private final ILSMComponent component;
private final IIndexBulkLoader invIndexBulkLoader;
- private boolean exceptionCaught = false;
+ private boolean cleanedUpArtifacts = false;
+ private boolean isEmptyComponent = true;
- public LSMInvertedIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws IndexException {
+ public LSMInvertedIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws IndexException, HyracksDataException {
+ if (checkIfEmptyIndex && !isEmptyIndex()) {
+ throw new IndexException("Cannot load an index that is not empty");
+ }
// Note that by using a flush target file name, we state that the
// new bulk loaded tree is "newer" than any other merged tree.
try {
component = createBulkLoadTarget();
- } catch (HyracksDataException e) {
- throw new TreeIndexException(e);
- } catch (IndexException e) {
- throw new TreeIndexException(e);
+ } catch (HyracksDataException | IndexException e) {
+ throw new IndexException(e);
}
invIndexBulkLoader = ((LSMInvertedIndexImmutableComponent) component).getInvIndex().createBulkLoader(
- fillFactor, verifyInput, numElementsHint);
+ fillFactor, verifyInput, numElementsHint, false);
}
@Override
public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
try {
invIndexBulkLoader.add(tuple);
- } catch (IndexException e) {
- handleException();
+ } catch (IndexException | HyracksDataException | RuntimeException e) {
+ cleanupArtifacts();
throw e;
- } catch (HyracksDataException e) {
- handleException();
- throw e;
- } catch (RuntimeException e) {
- handleException();
- throw e;
+ }
+ if (isEmptyComponent) {
+ isEmptyComponent = false;
}
}
- protected void handleException() throws HyracksDataException {
- exceptionCaught = true;
- ((LSMInvertedIndexImmutableComponent) component).getInvIndex().deactivate();
- ((LSMInvertedIndexImmutableComponent) component).getInvIndex().destroy();
- ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().deactivate();
- ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().destroy();
- ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().deactivate();
- ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().destroy();
+ protected void cleanupArtifacts() throws HyracksDataException {
+ if (!cleanedUpArtifacts) {
+ cleanedUpArtifacts = true;
+ ((LSMInvertedIndexImmutableComponent) component).getInvIndex().deactivate();
+ ((LSMInvertedIndexImmutableComponent) component).getInvIndex().destroy();
+ ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().deactivate();
+ ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().destroy();
+ ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().deactivate();
+ ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().destroy();
+ }
}
@Override
public void end() throws IndexException, HyracksDataException {
- if (!exceptionCaught) {
+ if (!cleanedUpArtifacts) {
invIndexBulkLoader.end();
+ if (isEmptyComponent) {
+ cleanupArtifacts();
+ } else {
+ lsmHarness.addBulkLoadedComponent(component);
+ }
}
- lsmHarness.addBulkLoadedComponent(component);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
index db6c337..825e5d3 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
@@ -26,7 +26,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
@@ -61,9 +60,8 @@
}
};
- public LSMInvertedIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- BTreeFactory btreeFactory, int ioDeviceId) {
- super(ioManager, fileMapProvider, file, null, ioDeviceId);
+ public LSMInvertedIndexFileManager(IFileMapProvider fileMapProvider, FileReference file, BTreeFactory btreeFactory) {
+ super(fileMapProvider, file, null);
this.btreeFactory = btreeFactory;
}
@@ -99,8 +97,8 @@
ArrayList<ComparableFileName> allDeletedKeysBTreeFiles = new ArrayList<ComparableFileName>();
ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
- // Gather files from the IODeviceHandle.
- cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
+ // Gather files.
+ cleanupAndGetValidFilesInternal(deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
@@ -108,9 +106,9 @@
}
// TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
- validateFiles(dev, deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
- validateFiles(dev, deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
- validateFiles(dev, deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+ validateFiles(deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
+ validateFiles(deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
+ validateFiles(deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
// Sanity check.
if (allDictBTreeFiles.size() != allInvListsFiles.size()
@@ -151,30 +149,30 @@
validComparableBloomFilterFiles.add(lastBloomFilter);
for (int i = 1; i < allDictBTreeFiles.size(); i++) {
- ComparableFileName currentRTree = allDictBTreeFiles.get(i);
- ComparableFileName currentBTree = allDictBTreeFiles.get(i);
+ ComparableFileName currentDeletedKeysBTree = allDeletedKeysBTreeFiles.get(i);
+ ComparableFileName CurrentDictBTree = allDictBTreeFiles.get(i);
ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
// Current start timestamp is greater than last stop timestamp.
- if (currentRTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
- && currentBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
+ if (currentDeletedKeysBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
+ && CurrentDictBTree.interval[0].compareTo(lastDictBTree.interval[1]) > 0
&& currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
- validComparableDictBTreeFiles.add(currentRTree);
- validComparableDeletedKeysBTreeFiles.add(currentBTree);
+ validComparableDictBTreeFiles.add(CurrentDictBTree);
+ validComparableDeletedKeysBTreeFiles.add(currentDeletedKeysBTree);
validComparableBloomFilterFiles.add(currentBloomFilter);
- lastDictBTree = currentRTree;
- lastDeletedKeysBTree = currentBTree;
+ lastDictBTree = CurrentDictBTree;
+ lastDeletedKeysBTree = currentDeletedKeysBTree;
lastBloomFilter = currentBloomFilter;
- } else if (currentRTree.interval[0].compareTo(lastDictBTree.interval[0]) >= 0
- && currentRTree.interval[1].compareTo(lastDictBTree.interval[1]) <= 0
- && currentBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[0]) >= 0
- && currentBTree.interval[1].compareTo(lastDeletedKeysBTree.interval[1]) <= 0
+ } else if (currentDeletedKeysBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[0]) >= 0
+ && currentDeletedKeysBTree.interval[1].compareTo(lastDeletedKeysBTree.interval[1]) <= 0
+ && CurrentDictBTree.interval[0].compareTo(lastDictBTree.interval[0]) >= 0
+ && CurrentDictBTree.interval[1].compareTo(lastDictBTree.interval[1]) <= 0
&& currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
&& currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0) {
// Invalid files are completely contained in last interval.
- File invalidRTreeFile = new File(currentRTree.fullPath);
- invalidRTreeFile.delete();
- File invalidBTreeFile = new File(currentBTree.fullPath);
- invalidBTreeFile.delete();
+ File invalidDeletedBTreeFile = new File(currentDeletedKeysBTree.fullPath);
+ invalidDeletedBTreeFile.delete();
+ File invalidDictBTreeFile = new File(CurrentDictBTree.fullPath);
+ invalidDictBTreeFile.delete();
File invalidBloomFilterFile = new File(currentBloomFilter.fullPath);
invalidBloomFilterFile.delete();
} else {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
index 9e5f75b..fd83bfb 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -18,6 +18,7 @@
import java.util.LinkedList;
import java.util.List;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -66,7 +67,7 @@
@Override
// TODO: Ignore opcallback for now.
- public void setOperation(IndexOperation newOp) {
+ public void setOperation(IndexOperation newOp) throws HyracksDataException {
reset();
switch (newOp) {
case INSERT:
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index b9f50a1..882d1a1 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -165,14 +165,7 @@
@Override
public void close() throws HyracksDataException {
- try {
- reset();
- accessorIndex = -1;
- } finally {
- if (harness != null) {
- harness.endSearch(opCtx);
- }
- }
+ reset();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 068031d..222f4de 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -163,7 +163,7 @@
@Override
public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback) {
+ ISearchOperationCallback searchCallback) throws HyracksDataException {
return new InMemoryInvertedIndexAccessor(this, new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories,
tokenizerFactory));
}
@@ -188,8 +188,8 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws IndexException {
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws IndexException {
throw new UnsupportedOperationException("Bulk load not supported by in-memory inverted index.");
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index fe8dfd8..7e9b483 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -43,7 +43,8 @@
protected InMemoryInvertedIndex index;
protected BTreeAccessor btreeAccessor;
- public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx) {
+ public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx)
+ throws HyracksDataException {
this.opCtx = opCtx;
this.index = index;
this.searcher = createSearcher();
@@ -110,7 +111,7 @@
throw new UnsupportedOperationException("Upsert not supported by in-memory inverted index.");
}
- protected IInvertedIndexSearcher createSearcher() {
+ protected IInvertedIndexSearcher createSearcher() throws HyracksDataException {
return new TOccurrenceSearcher(hyracksCtx, index);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index ae869906..eb06758 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -83,7 +83,7 @@
@Override
public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback) {
+ ISearchOperationCallback searchCallback) throws HyracksDataException {
return new PartitionedInMemoryInvertedIndexAccessor(this, new PartitionedInMemoryInvertedIndexOpContext(btree,
tokenCmpFactories, tokenizerFactory));
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
index 133df88..6b8bf00 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
@@ -15,17 +15,19 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
public class PartitionedInMemoryInvertedIndexAccessor extends InMemoryInvertedIndexAccessor {
- public PartitionedInMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx) {
+ public PartitionedInMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx)
+ throws HyracksDataException {
super(index, opCtx);
}
- protected IInvertedIndexSearcher createSearcher() {
+ protected IInvertedIndexSearcher createSearcher() throws HyracksDataException {
return new PartitionedTOccurrenceSearcher(hyracksCtx, index);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 5f75f8d..0ab2f4d 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -303,7 +303,7 @@
private final MultiComparator allCmp;
public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
- int startPageId, int fileId) throws IndexException, HyracksDataException {
+ boolean checkIfEmptyIndex, int startPageId, int fileId) throws IndexException, HyracksDataException {
this.verifyInput = verifyInput;
this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
this.invListCmp = MultiComparator.create(invListCmpFactories);
@@ -316,7 +316,8 @@
this.btreeTupleReference = new ArrayTupleReference();
this.lastTupleBuilder = new ArrayTupleBuilder(numTokenFields + numInvListKeys);
this.lastTuple = new ArrayTupleReference();
- this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint);
+ this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint,
+ checkIfEmptyIndex);
currentPageId = startPageId;
currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
currentPage.acquireWriteLatch();
@@ -467,7 +468,7 @@
private final IInvertedIndexSearcher searcher;
private final IIndexOperationContext opCtx = new OnDiskInvertedIndexOpContext(btree);
- public OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) {
+ public OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) throws HyracksDataException {
this.index = index;
this.searcher = new TOccurrenceSearcher(ctx, index);
}
@@ -535,7 +536,7 @@
@Override
public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback) {
+ ISearchOperationCallback searchCallback) throws HyracksDataException {
return new OnDiskInvertedIndexAccessor(this);
}
@@ -563,10 +564,11 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws IndexException {
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws IndexException {
try {
- return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, rootPageId, fileId);
+ return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+ rootPageId, fileId);
} catch (HyracksDataException e) {
throw new InvertedIndexException(e);
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index 2b2a174..6f7873b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -51,14 +51,14 @@
}
public class PartitionedOnDiskInvertedIndexAccessor extends OnDiskInvertedIndexAccessor {
- public PartitionedOnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) {
+ public PartitionedOnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) throws HyracksDataException {
super(index, new PartitionedTOccurrenceSearcher(ctx, index));
}
}
@Override
public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
- ISearchOperationCallback searchCallback) {
+ ISearchOperationCallback searchCallback) throws HyracksDataException {
return new PartitionedOnDiskInvertedIndexAccessor(this);
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 44d2b5d..2bbf1f3 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -68,7 +68,7 @@
protected final IObjectFactory<IInvertedListCursor> invListCursorFactory;
protected final ObjectCache<IInvertedListCursor> invListCursorCache;
- public AbstractTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+ public AbstractTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException {
this.ctx = ctx;
this.invListMerger = new InvertedListMerger(ctx, invIndex);
this.searchResult = new SearchResult(invIndex.getInvListTypeTraits(), ctx);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
index a6b825a..692a640 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -38,7 +38,7 @@
protected SearchResult prevSearchResult;
protected SearchResult newSearchResult;
- public InvertedListMerger(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+ public InvertedListMerger(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException {
this.invListCmp = MultiComparator.createIgnoreFieldLength(invIndex.getInvListCmpFactories());
this.prevSearchResult = new SearchResult(invIndex.getInvListTypeTraits(), ctx);
this.newSearchResult = new SearchResult(prevSearchResult);
@@ -72,11 +72,9 @@
int currentNumResults = prevSearchResult.getNumResults();
// Should we binary search the next list or should we sort-merge it?
if (currentNumResults * Math.log(numInvListElements) < currentNumResults + numInvListElements) {
- mergeSuffixListProbe(invListCursor, prevSearchResult, result, i, numInvLists,
- occurrenceThreshold);
+ mergeSuffixListProbe(invListCursor, prevSearchResult, result, i, numInvLists, occurrenceThreshold);
} else {
- mergeSuffixListScan(invListCursor, prevSearchResult, result, i, numInvLists,
- occurrenceThreshold);
+ mergeSuffixListScan(invListCursor, prevSearchResult, result, i, numInvLists, occurrenceThreshold);
}
}
invListCursor.unpinPages();
@@ -319,7 +317,7 @@
}
}
- public SearchResult createSearchResult() {
+ public SearchResult createSearchResult() throws HyracksDataException {
return new SearchResult(prevSearchResult);
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
index 18313c1..adad532 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -49,7 +49,8 @@
protected final ArrayList<IInvertedListCursor> cursorsOrderedByTokens = new ArrayList<IInvertedListCursor>();
protected final InvertedListPartitions partitions = new InvertedListPartitions();
- public PartitionedTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+ public PartitionedTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex)
+ throws HyracksDataException {
super(ctx, invIndex);
initHelperTuples();
}
@@ -89,19 +90,19 @@
if (partInvIndex.isEmpty()) {
return;
}
-
+
tokenizeQuery(searchPred);
short numQueryTokens = (short) queryTokenAccessor.getTupleCount();
IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
short numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
short numTokensUpperBound = searchModifier.getNumTokensUpperBound(numQueryTokens);
-
+
occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
if (occurrenceThreshold <= 0) {
throw new OccurrenceThresholdPanicException("Merge Threshold is <= 0. Failing Search.");
}
-
+
short maxCountPossible = numQueryTokens;
invListCursorCache.reset();
partitions.reset(numTokensLowerBound, numTokensUpperBound);
@@ -112,16 +113,16 @@
partitions, cursorsOrderedByTokens)) {
maxCountPossible--;
// No results possible.
- if (maxCountPossible < occurrenceThreshold) {
+ if (maxCountPossible < occurrenceThreshold) {
return;
}
}
}
-
+
ArrayList<IInvertedListCursor>[] partitionCursors = partitions.getPartitions();
short start = partitions.getMinValidPartitionIndex();
short end = partitions.getMaxValidPartitionIndex();
-
+
// Typically, we only enter this case for disk-based inverted indexes.
// TODO: This behavior could potentially lead to a deadlock if we cannot pin
// all inverted lists in memory, and are forced to wait for a page to get evicted
@@ -145,7 +146,7 @@
cursorsOrderedByTokens.get(i).pinPages();
}
}
-
+
// Process the partitions one-by-one.
for (int i = start; i <= end; i++) {
if (partitionCursors[i] == null) {
@@ -160,7 +161,7 @@
invListMerger.reset();
invListMerger.merge(partitionCursors[i], occurrenceThreshold, numPrefixLists, searchResult);
}
-
+
resultCursor.open(null, searchPred);
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
index b091195..c4056c5 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAccessor;
@@ -42,7 +43,7 @@
protected int currBufIdx;
protected int numResults;
- public SearchResult(ITypeTraits[] invListFields, IHyracksCommonContext ctx) {
+ public SearchResult(ITypeTraits[] invListFields, IHyracksCommonContext ctx) throws HyracksDataException {
typeTraits = new ITypeTraits[invListFields.length + 1];
int tmp = 0;
for (int i = 0; i < invListFields.length; i++) {
@@ -61,8 +62,10 @@
/**
* Initialize from other search-result object to share member instances except for result buffers.
+ *
+ * @throws HyracksDataException
*/
- public SearchResult(SearchResult other) {
+ public SearchResult(SearchResult other) throws HyracksDataException {
this.ctx = other.ctx;
this.appender = other.appender;
this.accessor = other.accessor;
@@ -102,7 +105,7 @@
}
}
- public void append(ITupleReference invListElement, int count) {
+ public void append(ITupleReference invListElement, int count) throws HyracksDataException {
ByteBuffer currentBuffer = buffers.get(currBufIdx);
if (!appender.hasSpace()) {
currBufIdx++;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
index 25552ea..348ef75 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
@@ -31,7 +31,7 @@
protected final ArrayList<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>();
- public TOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+ public TOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException {
super(ctx, invIndex);
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 17a6742..9d85f56 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -20,7 +20,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
@@ -120,21 +119,9 @@
IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+ IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
- return createLSMInvertedIndex(virtualBufferCache, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager, onDiskDir,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, 0);
- }
-
- public static LSMInvertedIndex createLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
- IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
- IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -147,8 +134,8 @@
bloomFilterKeyFields);
FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
- LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
- onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
+ LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(diskFileMapProvider,
+ onDiskDirFileRef, deletedKeysBTreeFactory);
IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
invListTypeTraits);
@@ -167,21 +154,9 @@
IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+ IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
- return createPartitionedLSMInvertedIndex(virtualBufferCache, diskFileMapProvider, invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager,
- onDiskDir, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, 0);
- }
-
- public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
- IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
- IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -194,8 +169,8 @@
bloomFilterKeyFields);
FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
- LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
- onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
+ LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(diskFileMapProvider,
+ onDiskDirFileRef, deletedKeysBTreeFactory);
IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
invListTypeTraits);
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index 169efc4..9fc8cea 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -21,7 +21,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
@@ -73,18 +72,17 @@
@Override
public ITreeIndex createIndexInstance() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
- return createLSMTree(virtualBufferCache, ctx.getIOManager(), file,
- opDesc.getStorageManager().getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
- treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(),
- btreeComparatorFactories, opTrackerFactory.getOperationTracker(ctx), valueProviderFactories,
- rtreePolicyType, linearizeCmpFactory, partition);
+ return createLSMTree(virtualBufferCache, file, opDesc.getStorageManager().getBufferCache(ctx), opDesc
+ .getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
+ treeOpDesc.getTreeIndexComparatorFactories(), btreeComparatorFactories,
+ opTrackerFactory.getOperationTracker(ctx), valueProviderFactories, rtreePolicyType, linearizeCmpFactory);
}
- protected abstract ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
- IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException;
+ protected abstract ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
+ throws HyracksDataException;
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index f8b6792..b47bc02 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -21,7 +21,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
@@ -62,18 +61,17 @@
}
@Override
- protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
- IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
+ protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
+ throws HyracksDataException {
try {
- return LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
- diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackProvider, linearizeCmpFactory,
- opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+ return LSMRTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+ typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ linearizeCmpFactory);
} catch (TreeIndexException e) {
throw new HyracksDataException(e);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
index 3163e3a..ea1b1e2 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -21,7 +21,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
@@ -49,17 +48,16 @@
}
@Override
- protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
- IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
+ protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
+ throws HyracksDataException {
try {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache, ioManager, file,
- diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
- valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
- linearizeCmpFactory, startIODeviceIndex);
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache, file, diskBufferCache,
+ diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+ rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
} catch (TreeIndexException e) {
throw new HyracksDataException(e);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index 632d462..e3e690b 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -185,6 +185,7 @@
mutableComponent.getRTree().clear();
mutableComponent.getBTree().clear();
+ mutableComponent.reset();
}
@Override
@@ -346,11 +347,7 @@
}
public boolean isEmptyIndex() throws HyracksDataException {
- return componentsRef.get().isEmpty()
- && mutableComponent.getBTree().isEmptyTree(
- mutableComponent.getBTree().getInteriorFrameFactory().createFrame())
- && mutableComponent.getRTree().isEmptyTree(
- mutableComponent.getRTree().getInteriorFrameFactory().createFrame());
+ return componentsRef.get().isEmpty() && !mutableComponent.isModified();
}
@Override
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 2013d74..0b71373 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -263,7 +263,7 @@
if (!isEmpty) {
rTreeTupleSorter.sort();
- rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L);
+ rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L, false);
cursor = rTreeTupleSorter;
try {
@@ -305,7 +305,7 @@
BTree diskBTree = component.getBTree();
// BulkLoad the tuples from the in-memory tree into the new disk BTree.
- IIndexBulkLoader bTreeBulkloader = diskBTree.createBulkLoader(1.0f, false, numBTreeTuples);
+ IIndexBulkLoader bTreeBulkloader = diskBTree.createBulkLoader(1.0f, false, numBTreeTuples, false);
IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numBTreeTuples,
bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
// scan the memory BTree
@@ -356,7 +356,7 @@
LSMRTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
mergeOp.getRTreeMergeTarget(), mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
- IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L);
+ IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L, false);
try {
while (cursor.hasNext()) {
@@ -400,59 +400,72 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
- return new LSMRTreeBulkLoader(fillLevel, verifyInput, numElementsHint);
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws TreeIndexException {
+ try {
+ return new LSMRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex);
+ } catch (HyracksDataException e) {
+ throw new TreeIndexException(e);
+ }
}
public class LSMRTreeBulkLoader implements IIndexBulkLoader {
private final ILSMComponent component;
private final IIndexBulkLoader bulkLoader;
+ private boolean cleanedUpArtifacts = false;
+ private boolean isEmptyComponent = true;
- public LSMRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
+ public LSMRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex)
+ throws TreeIndexException, HyracksDataException {
+ if (checkIfEmptyIndex && !isEmptyIndex()) {
+ throw new TreeIndexException("Cannot load an index that is not empty");
+ }
// Note that by using a flush target file name, we state that the
// new bulk loaded tree is "newer" than any other merged tree.
try {
component = createBulkLoadTarget();
- } catch (HyracksDataException e) {
- throw new TreeIndexException(e);
- } catch (IndexException e) {
+ } catch (HyracksDataException | IndexException e) {
throw new TreeIndexException(e);
}
bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
- numElementsHint);
+ numElementsHint, false);
}
@Override
public void add(ITupleReference tuple) throws HyracksDataException, IndexException {
try {
bulkLoader.add(tuple);
- } catch (IndexException e) {
- handleException();
+ } catch (IndexException | HyracksDataException | RuntimeException e) {
+ cleanupArtifacts();
throw e;
- } catch (HyracksDataException e) {
- handleException();
- throw e;
- } catch (RuntimeException e) {
- handleException();
- throw e;
+ }
+ if (isEmptyComponent) {
+ isEmptyComponent = false;
}
}
@Override
public void end() throws HyracksDataException, IndexException {
- bulkLoader.end();
- lsmHarness.addBulkLoadedComponent(component);
+ if (!cleanedUpArtifacts) {
+ bulkLoader.end();
+ if (isEmptyComponent) {
+ cleanupArtifacts();
+ } else {
+ lsmHarness.addBulkLoadedComponent(component);
+ }
+ }
}
- protected void handleException() throws HyracksDataException {
- ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
- ((LSMRTreeImmutableComponent) component).getRTree().destroy();
- ((LSMRTreeImmutableComponent) component).getBTree().deactivate();
- ((LSMRTreeImmutableComponent) component).getBTree().destroy();
- ((LSMRTreeImmutableComponent) component).getBloomFilter().deactivate();
- ((LSMRTreeImmutableComponent) component).getBloomFilter().destroy();
+ protected void cleanupArtifacts() throws HyracksDataException {
+ if (!cleanedUpArtifacts) {
+ cleanedUpArtifacts = true;
+ ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
+ ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+ ((LSMRTreeImmutableComponent) component).getBTree().deactivate();
+ ((LSMRTreeImmutableComponent) component).getBTree().destroy();
+ ((LSMRTreeImmutableComponent) component).getBloomFilter().deactivate();
+ ((LSMRTreeImmutableComponent) component).getBloomFilter().destroy();
+ }
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
index cfdf706..9502d5e 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
@@ -26,7 +26,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -53,10 +52,9 @@
}
};
- public LSMRTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory,
- int ioDeviceId) {
- super(ioManager, fileMapProvider, file, null, ioDeviceId);
+ public LSMRTreeFileManager(IFileMapProvider fileMapProvider, FileReference file,
+ TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory) {
+ super(fileMapProvider, file, null);
this.rtreeFactory = rtreeFactory;
this.btreeFactory = btreeFactory;
}
@@ -92,15 +90,15 @@
ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
- // Gather files from the IODeviceHandle.
- cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+ // Gather files.
+ cleanupAndGetValidFilesInternal(btreeFilter, btreeFactory, allBTreeFiles);
HashSet<String> btreeFilesSet = new HashSet<String>();
for (ComparableFileName cmpFileName : allBTreeFiles) {
int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
}
- validateFiles(dev, btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
- validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+ validateFiles(btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
+ validateFiles(btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
// Sanity check.
if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index cf3bbc6e..0be497a 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -273,7 +273,7 @@
bTreeTupleSorter.sort();
}
- IIndexBulkLoader rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L);
+ IIndexBulkLoader rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L, false);
LSMRTreeWithAntiMatterTuplesFlushCursor cursor = new LSMRTreeWithAntiMatterTuplesFlushCursor(rTreeTupleSorter,
bTreeTupleSorter, comparatorFields, linearizerArray);
cursor.open(null, null);
@@ -326,7 +326,7 @@
LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(),
null, null, true);
RTree mergedRTree = component.getRTree();
- IIndexBulkLoader bulkloader = mergedRTree.createBulkLoader(1.0f, false, 0L);
+ IIndexBulkLoader bulkloader = mergedRTree.createBulkLoader(1.0f, false, 0L, false);
try {
while (cursor.hasNext()) {
cursor.next();
@@ -363,9 +363,14 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
- return new LSMRTreeWithAntiMatterTuplesBulkLoader(fillLevel, verifyInput, numElementsHint);
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws TreeIndexException {
+ try {
+ return new LSMRTreeWithAntiMatterTuplesBulkLoader(fillLevel, verifyInput, numElementsHint,
+ checkIfEmptyIndex);
+ } catch (HyracksDataException e) {
+ throw new TreeIndexException(e);
+ }
}
private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
@@ -377,47 +382,56 @@
public class LSMRTreeWithAntiMatterTuplesBulkLoader implements IIndexBulkLoader {
private final ILSMComponent component;
private final IIndexBulkLoader bulkLoader;
+ private boolean cleanedUpArtifacts = false;
+ private boolean isEmptyComponent = true;
- public LSMRTreeWithAntiMatterTuplesBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
+ public LSMRTreeWithAntiMatterTuplesBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws TreeIndexException, HyracksDataException {
+ if (checkIfEmptyIndex && !isEmptyIndex()) {
+ throw new TreeIndexException("Cannot load an index that is not empty");
+ }
// Note that by using a flush target file name, we state that the
// new bulk loaded tree is "newer" than any other merged tree.
try {
component = createBulkLoadTarget();
- } catch (HyracksDataException e) {
- throw new TreeIndexException(e);
- } catch (IndexException e) {
+ } catch (HyracksDataException | IndexException e) {
throw new TreeIndexException(e);
}
bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
- numElementsHint);
+ numElementsHint, false);
}
@Override
public void add(ITupleReference tuple) throws HyracksDataException, IndexException {
try {
bulkLoader.add(tuple);
- } catch (IndexException e) {
- handleException();
+ } catch (IndexException | HyracksDataException | RuntimeException e) {
+ cleanupArtifacts();
throw e;
- } catch (HyracksDataException e) {
- handleException();
- throw e;
- } catch (RuntimeException e) {
- handleException();
- throw e;
+ }
+ if (isEmptyComponent) {
+ isEmptyComponent = false;
}
}
@Override
public void end() throws HyracksDataException, IndexException {
- bulkLoader.end();
- lsmHarness.addBulkLoadedComponent(component);
+ if (!cleanedUpArtifacts) {
+ bulkLoader.end();
+ if (isEmptyComponent) {
+ cleanupArtifacts();
+ } else {
+ lsmHarness.addBulkLoadedComponent(component);
+ }
+ }
}
- protected void handleException() throws HyracksDataException {
- ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
- ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+ protected void cleanupArtifacts() throws HyracksDataException {
+ if (!cleanedUpArtifacts) {
+ cleanedUpArtifacts = true;
+ ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
+ ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+ }
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
index f426dea..8ef143f 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
@@ -16,7 +16,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
@@ -24,8 +23,8 @@
public class LSMRTreeWithAntiMatterTuplesFileManager extends AbstractLSMIndexFileManager {
- public LSMRTreeWithAntiMatterTuplesFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider,
- FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int ioDeviceId) {
- super(ioManager, fileMapProvider, file, rtreeFactory, ioDeviceId);
+ public LSMRTreeWithAntiMatterTuplesFileManager(IFileMapProvider fileMapProvider, FileReference file,
+ TreeIndexFactory<? extends ITreeIndex> rtreeFactory) {
+ super(fileMapProvider, file, rtreeFactory);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 07c6815..50e5d09 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -19,7 +19,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
@@ -59,28 +58,13 @@
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class LSMRTreeUtils {
- public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
- IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
- RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory)
- throws TreeIndexException {
- return createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
- rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
- linearizeCmpFactory, 0);
- }
-
- public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
- IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
- RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
- int ioDeviceId) throws TreeIndexException {
+ public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+ IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+ double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILinearizeComparatorFactory linearizeCmpFactory) throws TreeIndexException {
LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
@@ -113,8 +97,8 @@
BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
bloomFilterKeyFields);
- ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(ioManager, diskFileMapProvider, file,
- diskRTreeFactory, diskBTreeFactory, ioDeviceId);
+ ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(diskFileMapProvider, file, diskRTreeFactory,
+ diskBTreeFactory);
LSMRTree lsmTree = new LSMRTree(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
@@ -124,26 +108,13 @@
}
public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
- IVirtualBufferCache virtualBufferCache, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+ IVirtualBufferCache virtualBufferCache, FileReference file, IBufferCache diskBufferCache,
+ IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory)
throws TreeIndexException {
- return createLSMTreeWithAntiMatterTuples(virtualBufferCache, ioManager, file, diskBufferCache,
- diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizerCmpFactory, 0);
- }
-
- public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
- IVirtualBufferCache virtualBufferCache, IIOManager ioManager, FileReference file,
- IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory,
- int ioDeviceId) throws TreeIndexException {
LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
@@ -178,8 +149,8 @@
IBinaryComparatorFactory[] linearizerArray = { linearizerCmpFactory,
btreeCmpFactories[btreeCmpFactories.length - 1] };
- ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(ioManager,
- diskFileMapProvider, file, diskRTreeFactory, ioDeviceId);
+ ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(diskFileMapProvider, file,
+ diskRTreeFactory);
LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(virtualBufferCache,
rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index 5454832..528c004 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -860,8 +860,8 @@
}
@Override
- public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
- throws TreeIndexException {
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws TreeIndexException {
// TODO: verifyInput currently does nothing.
try {
return new RTreeBulkLoader(fillFactor);
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
index e07a375..7ae2414 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
@@ -24,11 +24,11 @@
public LocalResource getResourceByName(String name) throws HyracksDataException;
- public void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException;
+ public void insert(LocalResource resource) throws HyracksDataException;
- public void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException;
+ public void deleteResourceById(long id) throws HyracksDataException;
- public void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException;
+ public void deleteResourceByName(String name) throws HyracksDataException;
public List<LocalResource> getAllResources() throws HyracksDataException;
}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
index f4fa440..63b71a3 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
@@ -37,7 +37,7 @@
}
@Override
- public synchronized void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException {
+ public synchronized void insert(LocalResource resource) throws HyracksDataException {
long id = resource.getResourceId();
if (id2ResourceMap.containsKey(id)) {
@@ -48,7 +48,7 @@
}
@Override
- public synchronized void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException {
+ public synchronized void deleteResourceById(long id) throws HyracksDataException {
LocalResource resource = id2ResourceMap.get(id);
if (resource == null) {
throw new HyracksDataException("Resource doesn't exist");
@@ -58,7 +58,7 @@
}
@Override
- public synchronized void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException {
+ public synchronized void deleteResourceByName(String name) throws HyracksDataException {
LocalResource resource = name2ResourceMap.get(name);
if (resource == null) {
throw new HyracksDataException("Resource doesn't exist");
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
index 5bc7f28..02ce7f1 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -85,7 +85,7 @@
private int blockingValue;
private int expectedAfterBlock;
- public SearchTask() {
+ public SearchTask() throws HyracksDataException {
this.cb = new SynchronizingSearchOperationCallback();
this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
this.cursor = accessor.createSearchCursor();
@@ -199,7 +199,7 @@
private final ArrayTupleBuilder builder;
private final ArrayTupleReference tuple;
- public InsertionTask() {
+ public InsertionTask() throws HyracksDataException {
this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
this.tuple = new ArrayTupleReference();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
index f36419c..54eac1e 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
@@ -62,9 +62,9 @@
ctx.getIndex().deactivate();
ctx.getIndex().destroy();
}
-
+
@Override
protected String getTestOpName() {
return "BulkLoad";
}
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index 5b8f7d9..eafeff2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -621,7 +621,7 @@
LOGGER.info("Bulk loading " + ins + " tuples");
}
long start = System.currentTimeMillis();
- IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins);
+ IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins, true);
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
for (int i = 0; i < ins; i++) {
@@ -691,7 +691,7 @@
treeIndex.activate();
// Load sorted records, and expect to fail at tuple i.
- IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins);
+ IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins, true);
for (int j = 0; j < ins; j++) {
if (j > i) {
fail("Bulk load failure test unexpectedly succeeded past tuple: " + j);
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
index 62fd98b..85c7fa5 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
@@ -19,6 +19,7 @@
import java.util.TreeSet;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
@@ -28,17 +29,17 @@
protected final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
- public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+ public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) throws HyracksDataException {
super(fieldSerdes, index);
}
public void upsertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
- if (checkTuples.contains(checkTuple)) {
+ if (checkTuples.contains(checkTuple)) {
checkTuples.remove(checkTuple);
}
checkTuples.add(checkTuple);
}
-
+
@Override
public TreeSet<CheckTuple> getCheckTuples() {
return checkTuples;
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
index 9f38d3a..3c8b740 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
@@ -35,7 +35,8 @@
protected final IIndexAccessor indexAccessor;
- public AbstractIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+ public AbstractIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+ throws HyracksDataException {
this.dataGen = dataGen;
this.opSelector = opSelector;
this.numBatches = numBatches;
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
index f56614e..65ed2ce 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
@@ -15,10 +15,11 @@
package edu.uci.ics.hyracks.storage.am.common;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
public interface IIndexTestWorkerFactory {
public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
- int numBatches);
+ int numBatches) throws HyracksDataException;
}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
index 0cb36d6..d8f9485 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
@@ -46,7 +46,7 @@
}
public long[] run(int numThreads, int numRepeats, int numOps, int batchSize) throws InterruptedException,
- TreeIndexException {
+ TreeIndexException, HyracksDataException {
int numBatches = numOps / batchSize;
int threadNumBatches = numBatches / numThreads;
if (threadNumBatches <= 0) {
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
index f91b1bf..339cd14 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
@@ -18,6 +18,7 @@
import java.util.Collection;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
@@ -31,7 +32,7 @@
protected final ArrayTupleReference tuple = new ArrayTupleReference();
protected final IIndexAccessor indexAccessor;
- public IndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+ public IndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) throws HyracksDataException {
this.fieldSerdes = fieldSerdes;
this.index = index;
this.indexAccessor = (IIndexAccessor) index.createAccessor(TestOperationCallback.INSTANCE,
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
index 852a0b6..b5742b5 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -243,7 +243,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
// Perform bulk load.
- IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples);
+ IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples, false);
int c = 1;
for (CheckTuple checkTuple : checkTuples) {
if (LOGGER.isLoggable(Level.INFO)) {
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
index e8d9c96..11adaa2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
@@ -26,11 +26,9 @@
public abstract class AbstractRTreeBulkLoadTest extends AbstractRTreeTestDriver {
private final RTreeTestUtils rTreeTestUtils;
- private final int bulkLoadRounds;
- public AbstractRTreeBulkLoadTest(int bulkLoadRounds, boolean testRstarPolicy) {
+ public AbstractRTreeBulkLoadTest(boolean testRstarPolicy) {
super(testRstarPolicy);
- this.bulkLoadRounds = bulkLoadRounds;
this.rTreeTestUtils = new RTreeTestUtils();
}
@@ -41,19 +39,19 @@
AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
ctx.getIndex().create();
ctx.getIndex().activate();
- for (int i = 0; i < bulkLoadRounds; i++) {
- // We assume all fieldSerdes are of the same type. Check the first
- // one to determine which field types to generate.
- if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
- rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
- } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
- rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
- }
- rTreeTestUtils.checkScan(ctx);
- rTreeTestUtils.checkDiskOrderScan(ctx);
- rTreeTestUtils.checkRangeSearch(ctx, key);
+ // We assume all fieldSerdes are of the same type. Check the first
+ // one to determine which field types to generate.
+ if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+ rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+ } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
+ rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
}
+
+ rTreeTestUtils.checkScan(ctx);
+ rTreeTestUtils.checkDiskOrderScan(ctx);
+ rTreeTestUtils.checkRangeSearch(ctx, key);
+
ctx.getIndex().deactivate();
ctx.getIndex().destroy();
}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 50ff154..44ab0d9 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -709,7 +709,7 @@
LOGGER.info("Bulk loading " + numInserts + " tuples");
}
long start = System.currentTimeMillis();
- IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts);
+ IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts, true);
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
index 135b306..77c0665 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
@@ -18,6 +18,7 @@
import java.util.Collection;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.util.HashMultiSet;
@@ -25,8 +26,9 @@
@SuppressWarnings("rawtypes")
public abstract class AbstractRTreeTestContext extends IndexTestContext<RTreeCheckTuple> {
private final HashMultiSet<RTreeCheckTuple> checkTuples = new HashMultiSet<RTreeCheckTuple>();
-
- public AbstractRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+
+ public AbstractRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex)
+ throws HyracksDataException {
super(fieldSerdes, treeIndex);
}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
index 756af98..9b77e23 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -21,6 +21,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializerContainer;
import edu.uci.ics.hyracks.api.messages.IMessageBroker;
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
public class TestNCApplicationContext implements INCApplicationContext {
private final IHyracksRootContext rootCtx;
@@ -29,9 +30,32 @@
private Serializable distributedState;
private Object appObject;
+ private final IMemoryManager mm;
+
public TestNCApplicationContext(IHyracksRootContext rootCtx, String nodeId) {
this.rootCtx = rootCtx;
this.nodeId = nodeId;
+ mm = new IMemoryManager() {
+ @Override
+ public long getMaximumMemory() {
+ return Long.MAX_VALUE;
+ }
+
+ @Override
+ public long getAvailableMemory() {
+ return Long.MAX_VALUE;
+ }
+
+ @Override
+ public void deallocate(long memory) {
+
+ }
+
+ @Override
+ public boolean allocate(long memory) {
+ return true;
+ }
+ };
}
@Override
@@ -67,14 +91,18 @@
@Override
public IMessageBroker getMessageBroker() {
- // TODO Auto-generated method stub
return null;
}
+ @Override
+ public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
@Override
- public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
- // TODO Auto-generated method stub
- return null;
+ public IMemoryManager getMemoryManager() {
+ return mm;
}
@Override
@@ -88,4 +116,4 @@
// TODO Auto-generated method stub
}
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
index db74413..262e21c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
@@ -40,7 +40,8 @@
private final ArrayTupleBuilder deleteTb;
private final ArrayTupleReference deleteTuple = new ArrayTupleReference();
- public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+ public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+ throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
btree = (BTree) index;
numKeyFields = btree.getComparatorFactories().length;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
index 039b1cf..a6d0425 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.btree.multithread;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
public class BTreeTestWorkerFactory implements IIndexTestWorkerFactory {
@Override
- public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+ int numBatches) throws HyracksDataException {
return new BTreeTestWorker(dataGen, opSelector, index, numBatches);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
index a3b82ad..7aa6f18 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
@@ -30,7 +31,7 @@
@SuppressWarnings("rawtypes")
public class BTreeTestContext extends OrderedIndexTestContext {
- public BTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+ public BTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
super(fieldSerdes, treeIndex);
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index ec4f7e2..e0a2a46 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -51,11 +51,10 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index b294f39..122b788 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -51,11 +51,10 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index afdd5cf..60e0492 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -34,11 +34,10 @@
@Override
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields) throws TreeIndexException {
- return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
- cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
+ bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index ab5fa1f..76a0206 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -51,11 +51,10 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index a10185d..639fd47 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -16,7 +16,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestUtils;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
@@ -40,10 +39,8 @@
@Override
protected boolean persistentStateExists() throws Exception {
// make sure all of the directories exist
- for (IODeviceHandle handle : harness.getIOManager().getIODevices()) {
- if (!new FileReference(handle, harness.getFileReference().getFile().getPath()).getFile().exists()) {
- return false;
- }
+ if (!new FileReference(harness.getFileReference().getFile()).getFile().exists()) {
+ return false;
}
return true;
}
@@ -56,11 +53,10 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, fieldSerdes.length, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
index = testCtx.getIndex();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index b84397c..e3c86df 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -50,11 +50,10 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
index 97b5eb4..f7aa7f4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -57,9 +57,15 @@
for (int i = 0; i < maxTreesToMerge; i++) {
for (int j = 0; j < i; j++) {
if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
- orderedIndexTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+ orderedIndexTestUtils.insertIntTuples(ctx, numTuplesToInsert, getRandom());
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ ctx.getIndex().deactivate();
+ ctx.getIndex().activate();
} else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
- orderedIndexTestUtils.bulkLoadStringTuples(ctx, numTuplesToInsert, getRandom());
+ orderedIndexTestUtils.insertStringTuples(ctx, numTuplesToInsert, getRandom());
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ ctx.getIndex().deactivate();
+ ctx.getIndex().activate();
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 0c5bf76..77b65ca 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -42,13 +42,13 @@
@Override
protected void createIndexInstance() throws Exception {
- index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index f0b230a..a7c8b81 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -52,15 +52,14 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
protected Random getRandom() {
return harness.getRandom();
}
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index e0db517..e15e80a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -49,13 +49,13 @@
@Override
protected void createIndexInstance() throws Exception {
- index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getIOOperationCallbackProvider());
}
@Override
@@ -94,7 +94,7 @@
private int expectedAfterBlock;
private int expectedTupleToBeLockedValue;
- public SearchTask() {
+ public SearchTask() throws HyracksDataException {
this.cb = new SynchronizingSearchOperationCallback();
this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
this.cursor = accessor.createSearchCursor();
@@ -210,7 +210,7 @@
private final ArrayTupleBuilder builder;
private final ArrayTupleReference tuple;
- public InsertionTask() {
+ public InsertionTask() throws HyracksDataException {
this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
this.tuple = new ArrayTupleReference();
@@ -222,11 +222,15 @@
try {
insertTaskStarted = true;
- // bulkload [101, 150] & [151, 200] as two separate disk components
+ // bulkload [101, 150] and then insert [151, 200] and make sure it reaches disk, thus we will have two separate disk components
// insert [50, 100] & [301, 350] to the in-memory component
// delete tuple 151
bulkloadIntTupleRange(101, 150);
- bulkloadIntTupleRange(151, 200);
+ insertIntTupleRange(151, 200);
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ index.deactivate();
+ index.activate();
+
insertIntTupleRange(50, 100);
insertIntTupleRange(301, 350);
int tupleTobeDeletedValue = 151;
@@ -270,7 +274,7 @@
throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
}
- IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin);
+ IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin, true);
for (int i = begin; i <= end; i++) {
TupleUtils.createIntegerTuple(builder, tuple, i);
bulkloader.add(tuple);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index 59b837c..db221a2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -51,11 +51,10 @@
@Override
protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
BTreeLeafFrameType leafType) throws Exception {
- return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index c5db1c1..9dfa713 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -50,11 +50,10 @@
@Override
protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
int[] bloomFilterKeyFields) throws TreeIndexException {
- return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
- cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
- harness.getIODeviceId());
+ return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
+ bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 80165d7..8a37b4e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -41,7 +41,8 @@
private final ArrayTupleBuilder deleteTb;
private final ArrayTupleReference deleteTuple = new ArrayTupleReference();
- public LSMBTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+ public LSMBTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+ throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
lsmBTree = (LSMBTree) index;
numKeyFields = lsmBTree.getComparatorFactories().length;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
index 6296fed..2015a66 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.btree.multithread;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
public class LSMBTreeTestWorkerFactory implements IIndexTestWorkerFactory {
@Override
- public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+ int numBatches) throws HyracksDataException {
return new LSMBTreeTestWorker(dataGen, opSelector, index, numBatches);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
index d364955..52cbddb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
@@ -37,7 +37,7 @@
public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception {
btree.create();
long start = System.currentTimeMillis();
- IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L);
+ IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L, true);
for (int i = 0; i < numBatches; i++) {
TupleBatch batch = dataGen.tupleBatchQueue.take();
for (int j = 0; j < batch.size(); j++) {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 1307212..fc6642c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -91,9 +91,9 @@
IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
inMemNumPages);
this.ioScheduler = SynchronousScheduler.INSTANCE;
- lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache, fmp, typeTraits,
- cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
- new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE, ioDeviceId);
+ lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fmp, typeTraits, cmpFactories,
+ bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
+ new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE);
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 5f8809b..ab78ee61 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -20,8 +20,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
@@ -38,7 +38,7 @@
@SuppressWarnings("rawtypes")
public final class LSMBTreeTestContext extends OrderedIndexTestContext {
- public LSMBTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+ public LSMBTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
super(fieldSerdes, treeIndex);
}
@@ -62,20 +62,20 @@
upsertCheckTuple(checkTuple, checkTuples);
}
- public static LSMBTreeTestContext create(IVirtualBufferCache virtualBufferCache, IOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ISerializerDeserializer[] fieldSerdes, int numKeyFields, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
+ public static LSMBTreeTestContext create(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+ int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
int[] bloomFilterKeyFields = new int[numKeyFields];
for (int i = 0; i < numKeyFields; ++i) {
bloomFilterKeyFields[i] = i;
}
- LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
- diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, ioDeviceId);
+ LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+ typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
+ ioScheduler, ioOpCallbackProvider);
LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree);
return testCtx;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 94aa841..e643cf9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -107,36 +107,36 @@
}
public void setUp() throws HyracksException {
- onDiskDir = "lsm_btree_" + simpleDateFormat.format(new Date()) + sep;
+ ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
+ onDiskDir = ioManager.getIODevices().get(ioDeviceId).getPath() + sep + "lsm_btree_"
+ + simpleDateFormat.format(new Date()) + sep;
file = new FileReference(new File(onDiskDir));
ctx = TestUtils.create(getHyracksFrameSize());
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
- ioManager = TestStorageManagerComponentHolder.getIOManager();
- ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
}
public void tearDown() throws HyracksDataException {
diskBufferCache.close();
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File dir = new File(dev.getPath(), onDiskDir);
- FilenameFilter filter = new FilenameFilter() {
- public boolean accept(File dir, String name) {
- return !name.startsWith(".");
- }
- };
- String[] files = dir.list(filter);
- if (files != null) {
- for (String fileName : files) {
- File file = new File(dir.getPath() + File.separator + fileName);
- file.delete();
- }
+ IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceId);
+ File dir = new File(dev.getPath(), onDiskDir);
+ FilenameFilter filter = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ return !name.startsWith(".");
}
- dir.delete();
+ };
+ String[] files = dir.list(filter);
+ if (files != null) {
+ for (String fileName : files) {
+ File file = new File(dir.getPath() + File.separator + fileName);
+ file.delete();
+ }
}
+ dir.delete();
}
public int getDiskPageSize() {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
index 19bf21f..e4dd974 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
@@ -21,8 +21,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -31,15 +29,15 @@
public class DummyLSMIndexFileManager extends AbstractLSMIndexFileManager {
- public DummyLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
- super(ioManager, fileMapProvider, file, treeFactory, ioDeviceId);
+ public DummyLSMIndexFileManager(IFileMapProvider fileMapProvider, FileReference file,
+ TreeIndexFactory<? extends ITreeIndex> treeFactory) {
+ super(fileMapProvider, file, treeFactory);
}
- protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
+ protected void cleanupAndGetValidFilesInternal(FilenameFilter filter,
TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
throws HyracksDataException, IndexException {
- File dir = new File(dev.getPath(), baseDir);
+ File dir = new File(baseDir);
String[] files = dir.list(filter);
for (String fileName : files) {
File file = new File(dir.getPath() + File.separator + fileName);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
index 5695eb8..d9fdeb3 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
@@ -61,7 +61,8 @@
TestStorageManagerComponentHolder.init(DEFAULT_PAGE_SIZE, DEFAULT_NUM_PAGES, DEFAULT_MAX_OPEN_FILES);
ioManager = TestStorageManagerComponentHolder.getIOManager();
fileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(null);
- baseDir = "lsm_tree" + simpleDateFormat.format(new Date()) + sep;
+ baseDir = ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath() + sep + "lsm_tree"
+ + simpleDateFormat.format(new Date()) + sep;
File f = new File(baseDir);
f.mkdirs();
file = new FileReference(f);
@@ -74,8 +75,7 @@
}
public void sortOrderTest(boolean testFlushFileName) throws InterruptedException, HyracksDataException {
- ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
- new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
+ ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(fileMapProvider, file, new DummyTreeFactory());
LinkedList<String> fileNames = new LinkedList<String>();
int numFileNames = 100;
@@ -115,8 +115,7 @@
}
public void cleanInvalidFilesTest(IOManager ioManager) throws InterruptedException, IOException, IndexException {
- ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
- new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
+ ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(fileMapProvider, file, new DummyTreeFactory());
fileManager.createDirs();
List<FileReference> flushFiles = new ArrayList<FileReference>();
@@ -190,7 +189,7 @@
// Make sure invalid files were removed from the IODevices.
ArrayList<String> remainingFiles = new ArrayList<String>();
- File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+ File dir = new File(baseDir);
FilenameFilter filter = new FilenameFilter() {
public boolean accept(File dir, String name) {
return !name.startsWith(".");
@@ -218,7 +217,7 @@
}
private void cleanDirs(IOManager ioManager) {
- File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+ File dir = new File(baseDir);
FilenameFilter filter = new FilenameFilter() {
public boolean accept(File dir, String name) {
return !name.startsWith(".");
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
index e254fba..ee09f2f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
public class LSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
public LSMInvertedIndexBulkLoadTest() {
- super(InvertedIndexType.LSM, true, 1);
+ super(InvertedIndexType.LSM, true);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
index 07afebe..748dd32 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
public class LSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
public LSMInvertedIndexInsertTest() {
- super(InvertedIndexType.LSM, false, 1);
+ super(InvertedIndexType.LSM, false);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
index 8c0c597..5115b7e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -34,7 +34,7 @@
private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
public LSMInvertedIndexMergeTest() {
- super(InvertedIndexType.LSM, true, 1);
+ super(InvertedIndexType.LSM, false);
}
@Override
@@ -48,11 +48,10 @@
for (int i = 0; i < maxTreesToMerge; i++) {
for (int j = 0; j < i; j++) {
- if (bulkLoad) {
- LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
- } else {
- LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
- }
+ LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ invIndex.deactivate();
+ invIndex.activate();
}
// Perform merge.
invIndexAccessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java
deleted file mode 100644
index 0d8f59c..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.invertedindex;
-
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
-
-public class LSMInvertedIndexMultiBulkLoadTest extends AbstractInvertedIndexLoadTest {
-
- public LSMInvertedIndexMultiBulkLoadTest() {
- super(InvertedIndexType.LSM, true, AccessMethodTestsConfig.LSM_INVINDEX_NUM_BULKLOAD_ROUNDS);
- }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
index 6c2da74..0f0345f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
public class PartitionedLSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
public PartitionedLSMInvertedIndexBulkLoadTest() {
- super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+ super(InvertedIndexType.PARTITIONED_LSM, true);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
index 7a93069..608f2fc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
public class PartitionedLSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
public PartitionedLSMInvertedIndexInsertTest() {
- super(InvertedIndexType.PARTITIONED_LSM, false, 1);
+ super(InvertedIndexType.PARTITIONED_LSM, false);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
index bccf27b..523557d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -34,7 +34,7 @@
private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
public PartitionedLSMInvertedIndexMergeTest() {
- super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+ super(InvertedIndexType.PARTITIONED_LSM, false);
}
@Override
@@ -48,11 +48,10 @@
for (int i = 0; i < maxTreesToMerge; i++) {
for (int j = 0; j < i; j++) {
- if (bulkLoad) {
- LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
- } else {
- LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
- }
+ LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ invIndex.deactivate();
+ invIndex.activate();
}
// Perform merge.
invIndexAccessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
deleted file mode 100644
index 71194f5..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.invertedindex;
-
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
-
-public class PartitionedLSMInvertedIndexMultiBulkLoadTest extends AbstractInvertedIndexLoadTest {
-
- public PartitionedLSMInvertedIndexMultiBulkLoadTest() {
- super(InvertedIndexType.PARTITIONED_LSM, true, AccessMethodTestsConfig.LSM_INVINDEX_NUM_BULKLOAD_ROUNDS);
- }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
index c3d6a16..8ccc203 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
@@ -29,12 +29,10 @@
public abstract class AbstractInvertedIndexLoadTest extends AbstractInvertedIndexTest {
protected final boolean bulkLoad;
- protected final int numRounds;
- public AbstractInvertedIndexLoadTest(InvertedIndexType invIndexType, boolean bulkLoad, int numRounds) {
+ public AbstractInvertedIndexLoadTest(InvertedIndexType invIndexType, boolean bulkLoad) {
super(invIndexType);
this.bulkLoad = bulkLoad;
- this.numRounds = numRounds;
}
protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
@@ -43,15 +41,13 @@
invIndex.create();
invIndex.activate();
- for (int i = 0; i < numRounds; i++) {
- if (bulkLoad) {
- LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
- } else {
- LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
- }
- validateAndCheckIndex(testCtx);
- runTinySearchWorkload(testCtx, tupleGen);
+ if (bulkLoad) {
+ LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+ } else {
+ LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
}
+ validateAndCheckIndex(testCtx);
+ runTinySearchWorkload(testCtx, tupleGen);
invIndex.deactivate();
invIndex.destroy();
@@ -59,7 +55,8 @@
@Test
public void wordTokensInvIndexTest() throws IOException, IndexException {
- LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness, invIndexType);
+ LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness,
+ invIndexType);
TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
runTest(testCtx, tupleGen);
}
@@ -74,7 +71,8 @@
@Test
public void ngramTokensInvIndexTest() throws IOException, IndexException {
- LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness, invIndexType);
+ LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness,
+ invIndexType);
TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
runTest(testCtx, tupleGen);
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 7407c24..7276464 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -108,7 +108,10 @@
}
public void setUp() throws HyracksException {
- onDiskDir = "lsm_invertedindex_" + simpleDateFormat.format(new Date()) + sep;
+ ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
+ onDiskDir = ioManager.getIODevices().get(ioDeviceId).getPath() + sep + "lsm_invertedindex_"
+ + simpleDateFormat.format(new Date()) + sep;
ctx = TestUtils.create(getHyracksFrameSize());
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -117,30 +120,27 @@
memPageSize, memNumPages));
virtualBufferCache.open();
virtualFreePageManager = new VirtualFreePageManager(memNumPages);
- ioManager = TestStorageManagerComponentHolder.getIOManager();
- ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
invIndexFileRef = ioManager.getIODevices().get(0).createFileReference(onDiskDir + invIndexFileName);
}
public void tearDown() throws HyracksDataException {
diskBufferCache.close();
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File dir = new File(dev.getPath(), onDiskDir);
- FilenameFilter filter = new FilenameFilter() {
- public boolean accept(File dir, String name) {
- return !name.startsWith(".");
- }
- };
- String[] files = dir.list(filter);
- if (files != null) {
- for (String fileName : files) {
- File file = new File(dir.getPath() + File.separator + fileName);
- file.delete();
- }
+ IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceId);
+ File dir = new File(dev.getPath(), onDiskDir);
+ FilenameFilter filter = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ return !name.startsWith(".");
}
- dir.delete();
+ };
+ String[] files = dir.list(filter);
+ if (files != null) {
+ for (String fileName : files) {
+ File file = new File(dir.getPath() + File.separator + fileName);
+ file.delete();
+ }
}
+ dir.delete();
virtualBufferCache.close();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
index bba0f36..8bc2817 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
public class InMemoryInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
public InMemoryInvertedIndexInsertTest() {
- super(InvertedIndexType.INMEMORY, false, 1);
+ super(InvertedIndexType.INMEMORY, false);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
index fa8eee0..46ea72c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
public class PartitionedInMemoryInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
public PartitionedInMemoryInvertedIndexInsertTest() {
- super(InvertedIndexType.PARTITIONED_INMEMORY, false, 1);
+ super(InvertedIndexType.PARTITIONED_INMEMORY, false);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index b4875a8..e1570af 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -50,7 +50,7 @@
new ConjunctiveSearchModifier(), new JaccardSearchModifier(0.8f), new JaccardSearchModifier(0.5f) };
public LSMInvertedIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
- int numBatches) {
+ int numBatches) throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
invIndex = (LSMInvertedIndex) index;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
index 6964f3d..c211edb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.multithread;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
public class LSMInvertedIndexWorkerFactory implements IIndexTestWorkerFactory {
@Override
- public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+ int numBatches) throws HyracksDataException {
return new LSMInvertedIndexTestWorker(dataGen, opSelector, index, numBatches);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
index fd90ad4..776b193 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
public class OnDiskInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
public OnDiskInvertedIndexBulkLoadTest() {
- super(InvertedIndexType.ONDISK, true, 1);
+ super(InvertedIndexType.ONDISK, true);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
index ad2b93a..6b9890d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
public class PartitionedOnDiskInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
public PartitionedOnDiskInvertedIndexBulkLoadTest() {
- super(InvertedIndexType.PARTITIONED_ONDISK, true, 1);
+ super(InvertedIndexType.PARTITIONED_ONDISK, true);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 119226e..05dc1b5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -63,7 +63,7 @@
public LSMInvertedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index,
IBinaryTokenizerFactory tokenizerFactory, InvertedIndexType invIndexType,
- InvertedIndexTokenizingTupleIterator indexTupleIter) {
+ InvertedIndexTokenizingTupleIterator indexTupleIter) throws HyracksDataException {
super(fieldSerdes, index);
invIndex = (IInvertedIndex) index;
this.tokenizerFactory = tokenizerFactory;
@@ -97,7 +97,7 @@
public static LSMInvertedIndexTestContext create(LSMInvertedIndexTestHarness harness,
ISerializerDeserializer[] fieldSerdes, int tokenFieldCount, IBinaryTokenizerFactory tokenizerFactory,
- InvertedIndexType invIndexType) throws IndexException {
+ InvertedIndexType invIndexType) throws IndexException, HyracksDataException {
ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] allCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
@@ -147,19 +147,19 @@
case LSM: {
invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getVirtualBufferCache(),
harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
- harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getIOOperationCallbackProvider());
break;
}
case PARTITIONED_LSM: {
invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getVirtualBufferCache(),
harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
- harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getIOOperationCallbackProvider());
break;
}
default: {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index fcbf133..c4e1ac5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -205,7 +205,7 @@
ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
// Use the expected index to bulk-load the actual index.
- IIndexBulkLoader bulkLoader = testCtx.getIndex().createBulkLoader(1.0f, false, numDocs);
+ IIndexBulkLoader bulkLoader = testCtx.getIndex().createBulkLoader(1.0f, false, numDocs, true);
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(testCtx.getFieldSerdes().length);
ArrayTupleReference tuple = new ArrayTupleReference();
Iterator<CheckTuple> checkTupleIter = tmpMemIndex.iterator();
@@ -493,7 +493,6 @@
int[] fieldPermutation = new int[] { 0 };
PermutingTupleReference searchDocument = new PermutingTupleReference(fieldPermutation);
- IIndexCursor resultCursor = accessor.createSearchCursor();
int numQueries = numDocQueries + numRandomQueries;
for (int i = 0; i < numQueries; i++) {
// If number of documents in the corpus is less than numDocQueries, then replace the remaining ones with random queries.
@@ -511,7 +510,7 @@
searchPred.setQueryTuple(searchDocument);
searchPred.setQueryFieldIndex(0);
- resultCursor.reset();
+ IIndexCursor resultCursor = accessor.createSearchCursor();
boolean panic = false;
try {
accessor.search(resultCursor, searchPred);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index b9f652a..3fcb4e5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -35,7 +35,7 @@
public class LSMRTreeBulkLoadTest extends AbstractRTreeBulkLoadTest {
public LSMRTreeBulkLoadTest() {
- super(1, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+ super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
}
private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
@@ -54,12 +54,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+ numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index c4c5d00..467916e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -54,12 +54,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+ numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 0778269..a35170c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -37,13 +37,12 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
- rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
+ return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
+ btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
}
@Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 33e0bca..44da0c0 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -54,12 +54,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+ numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index 5da1fe0..b97bebb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -16,7 +16,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
@@ -50,10 +49,8 @@
@Override
protected boolean persistentStateExists() throws Exception {
// make sure all of the directories exist
- for (IODeviceHandle handle : harness.getIOManager().getIODevices()) {
- if (!new FileReference(handle, harness.getFileReference().getFile().getPath()).getFile().exists()) {
- return false;
- }
+ if (!new FileReference(harness.getFileReference().getFile()).getFile().exists()) {
+ return false;
}
return true;
}
@@ -66,11 +63,10 @@
@Override
public void setup() throws Exception {
harness.setUp();
- testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+ numKeys, RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
index = testCtx.getIndex();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 42fde48..b13ce4d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -53,12 +53,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+ numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
index 4cde8d2..18528c4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -59,9 +59,15 @@
for (int i = 0; i < maxTreesToMerge; i++) {
for (int j = 0; j < i; j++) {
if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
- rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+ rTreeTestUtils.insertIntTuples(ctx, numTuplesToInsert, getRandom());
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ ctx.getIndex().deactivate();
+ ctx.getIndex().activate();
} else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
- rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
+ rTreeTestUtils.insertDoubleTuples(ctx, numTuplesToInsert, getRandom());
+ // Deactivate and the re-activate the index to force it flush its in memory component
+ ctx.getIndex().deactivate();
+ ctx.getIndex().activate();
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
deleted file mode 100644
index 6796546..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.rtree;
-
-import java.util.Random;
-
-import org.junit.After;
-import org.junit.Before;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestContext;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeBulkLoadTest;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
-import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-
-@SuppressWarnings("rawtypes")
-public class LSMRTreeMultiBulkLoadTest extends AbstractRTreeBulkLoadTest {
-
- public LSMRTreeMultiBulkLoadTest() {
- super(AccessMethodTestsConfig.LSM_RTREE_BULKLOAD_ROUNDS, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
- }
-
- private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
-
- @Before
- public void setUp() throws HyracksException {
- harness.setUp();
- }
-
- @After
- public void tearDown() throws HyracksDataException {
- harness.tearDown();
- }
-
- @Override
- protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
- IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
- throws Exception {
- return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
- }
-
- @Override
- protected Random getRandom() {
- return harness.getRandom();
- }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index cf33d05..2baf485 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -35,7 +35,7 @@
public class LSMRTreeWithAntiMatterTuplesBulkLoadTest extends AbstractRTreeBulkLoadTest {
public LSMRTreeWithAntiMatterTuplesBulkLoadTest() {
- super(1, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+ super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
}
private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
@@ -54,11 +54,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 7be02f3..9c440cc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -54,11 +54,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 71d6c1c..909d0cd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -37,12 +37,12 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(), harness.getIOManager(),
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
}
@Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index ad502c7..6b35192 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -54,11 +54,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index 44c1966..6dcbdf2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -16,7 +16,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
@@ -50,10 +49,8 @@
@Override
protected boolean persistentStateExists() throws Exception {
// make sure all of the directories exist
- for (IODeviceHandle handle : harness.getIOManager().getIODevices()) {
- if (!new FileReference(handle, harness.getFileReference().getFile().getPath()).getFile().exists()) {
- return false;
- }
+ if (!new FileReference(harness.getFileReference().getFile()).getFile().exists()) {
+ return false;
}
return true;
}
@@ -67,10 +64,9 @@
public void setup() throws Exception {
harness.setUp();
testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
- harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
- harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
- harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+ fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
index = testCtx.getIndex();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index 16a9679..dc81fc3 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -53,11 +53,10 @@
protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+ return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
deleted file mode 100644
index c91948e..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.rtree;
-
-import java.util.Random;
-
-import org.junit.After;
-import org.junit.Before;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeWithAntiMatterTuplesTestContext;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeBulkLoadTest;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
-import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-
-@SuppressWarnings("rawtypes")
-public class LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest extends AbstractRTreeBulkLoadTest {
-
- public LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest() {
- super(AccessMethodTestsConfig.LSM_RTREE_BULKLOAD_ROUNDS, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
- }
-
- private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
-
- @Before
- public void setUp() throws HyracksException {
- harness.setUp();
- }
-
- @After
- public void tearDown() throws HyracksDataException {
- harness.tearDown();
- }
-
- @Override
- protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
- IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
- throws Exception {
- return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
- fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
- }
-
- @Override
- protected Random getRandom() {
- return harness.getRandom();
- }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
index 8bf2edd..710d181 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
@@ -36,7 +36,7 @@
protected final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
public AbstractLSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
- int numBatches) {
+ int numBatches) throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
lsmRTree = (ITreeIndex) index;
numFields = lsmRTree.getFieldCount();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 8c6c305..850d540 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -57,13 +57,12 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
- harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
- rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
+ return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+ harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
+ btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index 09de3a3..b3fddc8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -39,7 +39,8 @@
private final ArrayTupleBuilder rearrangedTb;
private final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
- public LSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+ public LSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+ throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
lsmRTree = (LSMRTree) index;
numFields = lsmRTree.getFieldCount();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
index 2b2d72c..2df1747 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.multithread;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
public class LSMRTreeTestWorkerFactory implements IIndexTestWorkerFactory {
@Override
- public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+ int numBatches) throws HyracksDataException {
return new LSMRTreeTestWorker(dataGen, opSelector, index, numBatches);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index d9e4658..4a6b462 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -57,12 +57,12 @@
protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType) throws TreeIndexException {
- return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(), harness.getIOManager(),
+ return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index c27158f..12d7742 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -31,7 +31,7 @@
public class LSMRTreeWithAntiMatterTuplesTestWorker extends AbstractLSMRTreeTestWorker {
public LSMRTreeWithAntiMatterTuplesTestWorker(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ IIndex index, int numBatches) throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
index 67d9c27..cf3ac02 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.rtree.multithread;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
public class LSMRTreeWithAntiMatterTuplesTestWorkerFactory implements IIndexTestWorkerFactory {
@Override
- public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+ int numBatches) throws HyracksDataException {
return new LSMRTreeWithAntiMatterTuplesTestWorker(dataGen, opSelector, index, numBatches);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index fe2cc9e..8c59e31 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -20,8 +20,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -41,7 +41,7 @@
@SuppressWarnings("rawtypes")
public final class LSMRTreeTestContext extends AbstractRTreeTestContext {
- public LSMRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+ public LSMRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
super(fieldSerdes, treeIndex);
}
@@ -66,22 +66,21 @@
return lsmTree.getComparatorFactories();
}
- public static LSMRTreeTestContext create(IVirtualBufferCache virtualBufferCache, IOManager ioManager,
- FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
- ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
- int numKeyFields, RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
+ public static LSMRTreeTestContext create(IVirtualBufferCache virtualBufferCache, FileReference file,
+ IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+ IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
+ double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+ throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
- LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
- diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
- ioDeviceId);
+ LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+ typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
return testCtx;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 3382318..118b1bc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -104,35 +104,34 @@
}
public void setUp() throws HyracksException {
- onDiskDir = "lsm_rtree_" + simpleDateFormat.format(new Date()) + sep;
+ ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
+ onDiskDir = ioManager.getIODevices().get(ioDeviceId).getPath() + sep + "lsm_rtree_"
+ + simpleDateFormat.format(new Date()) + sep;
file = new FileReference(new File(onDiskDir));
ctx = TestUtils.create(getHyracksFrameSize());
TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
- ioManager = TestStorageManagerComponentHolder.getIOManager();
- ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
}
public void tearDown() throws HyracksDataException {
diskBufferCache.close();
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File dir = new File(dev.getPath(), onDiskDir);
- FilenameFilter filter = new FilenameFilter() {
- public boolean accept(File dir, String name) {
- return !name.startsWith(".");
- }
- };
- String[] files = dir.list(filter);
- if (files != null) {
- for (String fileName : files) {
- File file = new File(dir.getPath() + File.separator + fileName);
- file.delete();
- }
+ IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceId);
+ File dir = new File(dev.getPath(), onDiskDir);
+ FilenameFilter filter = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ return !name.startsWith(".");
}
- dir.delete();
+ };
+ String[] files = dir.list(filter);
+ if (files != null) {
+ for (String fileName : files) {
+ File file = new File(dir.getPath() + File.separator + fileName);
+ file.delete();
+ }
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index b1f4d8f..95716f6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -20,8 +20,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -41,7 +41,8 @@
@SuppressWarnings("rawtypes")
public final class LSMRTreeWithAntiMatterTuplesTestContext extends AbstractRTreeTestContext {
- public LSMRTreeWithAntiMatterTuplesTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+ public LSMRTreeWithAntiMatterTuplesTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex)
+ throws HyracksDataException {
super(fieldSerdes, treeIndex);
}
@@ -67,21 +68,20 @@
}
public static LSMRTreeWithAntiMatterTuplesTestContext create(IVirtualBufferCache virtualBufferCache,
- IOManager ioManager, FileReference file, IBufferCache diskBufferCache,
- IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
- IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
+ FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+ ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
+ int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache,
- ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories,
- btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
- ioDeviceId);
+ file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
+ valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
lsmTree);
return testCtx;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java
index 7f42223..4b68ddc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java
@@ -32,7 +32,7 @@
public class RTreeBulkLoadTest extends AbstractRTreeBulkLoadTest {
public RTreeBulkLoadTest() {
- super(1, AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
+ super(AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
}
private final RTreeTestHarness harness = new RTreeTestHarness();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
index cf118a3..bc5d331 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
@@ -38,7 +38,8 @@
private final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
private final ArrayTupleBuilder rearrangedTb;
- public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+ public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+ throws HyracksDataException {
super(dataGen, opSelector, index, numBatches);
rtree = (RTree) index;
numFields = rtree.getFieldCount();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java
index a1f2155..4d06601 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.rtree.multithread;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
public class RTreeTestWorkerFactory implements IIndexTestWorkerFactory {
@Override
- public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
- IIndex index, int numBatches) {
+ public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+ int numBatches) throws HyracksDataException {
return new RTreeTestWorker(dataGen, opSelector, index, numBatches);
}
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java
index 97ac221..5968fe9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -32,7 +33,7 @@
@SuppressWarnings("rawtypes")
public class RTreeTestContext extends AbstractRTreeTestContext {
- public RTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+ public RTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
super(fieldSerdes, treeIndex);
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
index 7c91c02..4175078 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
@@ -420,7 +420,7 @@
private M allocateMessage() {
M message;
if (usedMessage < msgPool.size()) {
- message = msgPool.get(usedEdge);
+ message = msgPool.get(usedMessage);
usedMessage++;
} else {
message = BspUtils.<M> createMessageValue(getContext().getConfiguration());
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexPartitioner.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexPartitioner.java
new file mode 100644
index 0000000..f51ad88
--- /dev/null
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexPartitioner.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.api.graph;
+
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Users can extend this class to implement the desired vertex partitioning behavior.
+ *
+ * @author yingyib
+ */
+@SuppressWarnings("rawtypes")
+public abstract class VertexPartitioner<I extends WritableComparable> {
+
+ /**
+ * @param vertexId
+ * The input vertex id.
+ * @param nPartitions
+ * The total number of partitions.
+ * @return The partition id.
+ */
+ public abstract int getPartitionId(I vertexId, int nPartitions);
+
+}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
index 81472aa..4cddaf0 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
@@ -24,6 +24,7 @@
import edu.uci.ics.pregelix.api.graph.MessageCombiner;
import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
import edu.uci.ics.pregelix.api.io.VertexInputFormat;
import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
@@ -59,6 +60,8 @@
public static final String FINAL_AGGREGATE_VALUE_CLASS = "pregelix.finalAggregateValueClass";
/** The normalized key computer class */
public static final String NMK_COMPUTER_CLASS = "pregelix.nmkComputerClass";
+ /** The partitioner class */
+ public static final String PARTITIONER_CLASS = "pregelix.partitionerClass";
/** num of vertices */
public static final String NUM_VERTICE = "pregelix.numVertices";
/** num of edges */
@@ -69,6 +72,8 @@
public static final String JOB_ID = "pregelix.jobid";
/** frame size */
public static final String FRAME_SIZE = "pregelix.framesize";
+ /** update intensive */
+ public static final String UPDATE_INTENSIVE = "pregelix.updateIntensive";
/**
* Constructor that will instantiate the configuration
@@ -178,4 +183,22 @@
final public void setNoramlizedKeyComputerClass(Class<?> nkcClass) {
getConfiguration().setClass(NMK_COMPUTER_CLASS, nkcClass, NormalizedKeyComputer.class);
}
+
+ /**
+ * Set the vertex partitioner class
+ *
+ * @param partitionerClass
+ */
+ final public void setVertexPartitionerClass(Class<?> partitionerClass) {
+ getConfiguration().setClass(PARTITIONER_CLASS, partitionerClass, VertexPartitioner.class);
+ }
+
+ /**
+ * Indicate if the job needs to do a lot of graph mutations or variable size updates
+ *
+ * @param updateHeavyFlag
+ */
+ final public void setMutationOrVariableSizedUpdateHeavy(boolean variableSizedUpdateHeavyFlag) {
+ getConfiguration().setBoolean(UPDATE_INTENSIVE, variableSizedUpdateHeavyFlag);
+ }
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index 6bac923..03c37dc 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -25,6 +25,7 @@
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
import edu.uci.ics.pregelix.api.io.VertexInputFormat;
import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -156,7 +157,7 @@
}
/**
- * Create a global aggregator class
+ * Create a global aggregator object
*
* @param conf
* Configuration to check
@@ -391,9 +392,9 @@
try {
return aggregateValueClass.newInstance();
} catch (InstantiationException e) {
- throw new IllegalArgumentException("createMessageValue: Failed to instantiate", e);
+ throw new IllegalArgumentException("createPartialAggregateValue: Failed to instantiate", e);
} catch (IllegalAccessException e) {
- throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
+ throw new IllegalArgumentException("createPartialAggregateValue: Illegally accessed", e);
}
}
@@ -415,9 +416,9 @@
}
return instance;
} catch (InstantiationException e) {
- throw new IllegalArgumentException("createMessageValue: Failed to instantiate", e);
+ throw new IllegalArgumentException("createPartialCombineValue: Failed to instantiate", e);
} catch (IllegalAccessException e) {
- throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
+ throw new IllegalArgumentException("createPartialCombineValue: Illegally accessed", e);
}
}
@@ -433,13 +434,46 @@
try {
return aggregateValueClass.newInstance();
} catch (InstantiationException e) {
- throw new IllegalArgumentException("createMessageValue: Failed to instantiate", e);
+ throw new IllegalArgumentException("createAggregateValue: Failed to instantiate", e);
} catch (IllegalAccessException e) {
- throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
+ throw new IllegalArgumentException("createAggregateValue: Illegally accessed", e);
}
}
/**
+ * Create a user aggregate value
+ *
+ * @param conf
+ * Configuration to check
+ * @return Instantiated user aggregate value
+ */
+ @SuppressWarnings("rawtypes")
+ public static VertexPartitioner createVertexPartitioner(Configuration conf) {
+ Class<? extends VertexPartitioner> vertexPartitionerClass = getVertexPartitionerClass(conf);
+ try {
+ return vertexPartitionerClass.newInstance();
+ } catch (InstantiationException e) {
+ throw new IllegalArgumentException("createVertexPartitioner: Failed to instantiate", e);
+ } catch (IllegalAccessException e) {
+ throw new IllegalArgumentException("createVertexPartitioner: Illegally accessed", e);
+ }
+ }
+
+ /**
+ * Get the user's subclassed vertex partitioner class.
+ *
+ * @param conf
+ * Configuration to check
+ * @return The user defined vertex partitioner class
+ */
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ public static <V extends VertexPartitioner> Class<V> getVertexPartitionerClass(Configuration conf) {
+ if (conf == null)
+ conf = defaultConf;
+ return (Class<V>) conf.getClass(PregelixJob.PARTITIONER_CLASS, null, VertexPartitioner.class);
+ }
+
+ /**
* Get the job configuration parameter whether the vertex states will increase dynamically
*
* @param conf
@@ -460,4 +494,14 @@
public static int getFrameSize(Configuration conf) {
return conf.getInt(PregelixJob.FRAME_SIZE, -1);
}
+
+ /**
+ * Should the job use LSM or B-tree to store vertices
+ *
+ * @param conf
+ * @return
+ */
+ public static boolean useLSM(Configuration conf) {
+ return conf.getBoolean(PregelixJob.UPDATE_INTENSIVE, false);
+ }
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultVertexPartitioner.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultVertexPartitioner.java
new file mode 100644
index 0000000..263ec65
--- /dev/null
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultVertexPartitioner.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.api.util;
+
+import org.apache.hadoop.io.WritableComparable;
+
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
+
+/**
+ * The deafult vertex partitioner which use the hashcode of the vertex id to determine the partition
+ * of the vertex.
+ *
+ * @author yingyib
+ */
+@SuppressWarnings("rawtypes")
+public class DefaultVertexPartitioner<I extends WritableComparable> extends VertexPartitioner<I> {
+
+ @Override
+ public int getPartitionId(I vertexId, int nPartitions) {
+ return vertexId.hashCode() % nPartitions;
+ }
+
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
index c0bbafd..1600ab5 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
@@ -45,4 +45,15 @@
throw new HyracksDataException(e);
}
}
+
+ @Override
+ public Configuration createConfiguration() throws HyracksDataException{
+ try {
+ Configuration conf = new Configuration();
+ SerDeUtils.deserialize(conf, data);
+ return conf;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index fcdf654..478ac07 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -53,15 +53,22 @@
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
import edu.uci.ics.pregelix.api.graph.MessageCombiner;
import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
import edu.uci.ics.pregelix.api.io.VertexInputFormat;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -80,8 +87,10 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.bootstrap.IndexLifeCycleManagerProvider;
import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
+import edu.uci.ics.pregelix.runtime.bootstrap.VirtualBufferCacheProvider;
import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.VertexPartitionComputerFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
public abstract class JobGen implements IJobGen {
@@ -166,10 +175,12 @@
comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
.getClass());
+ int[] keyFields = new int[1];
+ keyFields[0] = 0;
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
TreeIndexCreateOperatorDescriptor btreeCreate = new TreeIndexCreateOperatorDescriptor(spec,
- storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- new BTreeDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
+ storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+ keyFields, getIndexDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, btreeCreate);
spec.setFrameSize(frameSize);
@@ -229,16 +240,15 @@
typeTraits[0] = new TypeTraits(false);
typeTraits[1] = new TypeTraits(false);
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, new BTreeDataflowHelperFactory(),
- NoOpOperationCallbackFactory.INSTANCE);
+ storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+ sortFields, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, false,
+ getIndexDataflowHelperFactory(), NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
/**
* connect operator descriptors
*/
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sorter, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
spec.setFrameSize(frameSize);
@@ -313,8 +323,7 @@
/**
* connect operator descriptors
*/
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
comparatorFactories), sorter, 0, writer, 0);
@@ -358,7 +367,7 @@
typeTraits[1] = new TypeTraits(false);
BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
+ null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -381,8 +390,7 @@
*/
int[] sortFields = new int[1];
sortFields[0] = 0;
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
comparatorFactories), scanner, 0, writer, 0);
@@ -428,7 +436,7 @@
BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
+ null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -459,7 +467,7 @@
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, indexName);
IndexDropOperatorDescriptor drop = new IndexDropOperatorDescriptor(spec, storageManagerInterface,
- lcManagerProvider, fileSplitProvider, new BTreeDataflowHelperFactory());
+ lcManagerProvider, fileSplitProvider, getIndexDataflowHelperFactory());
ClusterConfig.setLocationConstraint(spec, drop);
spec.addRoot(drop);
@@ -467,6 +475,28 @@
return spec;
}
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ protected ITuplePartitionComputerFactory getVertexPartitionComputerFactory() {
+ IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+ Class<? extends VertexPartitioner> partitionerClazz = BspUtils.getVertexPartitionerClass(conf);
+ if (partitionerClazz != null) {
+ return new VertexPartitionComputerFactory(confFactory);
+ } else {
+ return new VertexIdPartitionComputerFactory(new WritableSerializerDeserializerFactory(
+ BspUtils.getVertexIndexClass(conf)));
+ }
+ }
+
+ protected IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
+ if (BspUtils.useLSM(conf)) {
+ return new LSMBTreeDataflowHelperFactory(new VirtualBufferCacheProvider(), new ConstantMergePolicyProvider(
+ 3), NoOpOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+ NoOpIOOperationCallback.INSTANCE, 0.01);
+ } else {
+ return new BTreeDataflowHelperFactory();
+ }
+ }
+
/** generate non-first iteration job */
protected abstract JobSpecification generateNonFirstIteration(int iteration) throws HyracksException;
@@ -476,4 +506,4 @@
/** generate clean-up job */
public abstract JobSpecification[] generateCleanup() throws HyracksException;
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index d9a24bc..db6c2c8 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -35,14 +35,9 @@
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -59,11 +54,11 @@
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.TreeIndexBulkReLoadOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -72,8 +67,6 @@
import edu.uci.ics.pregelix.runtime.touchpoint.PostSuperStepRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
public class JobGenInnerJoin extends JobGen {
@@ -135,12 +128,11 @@
vertexClass.getName());
RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
- BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+ TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 6,
- new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete, rdFinal);
+ getIndexDataflowHelperFactory(), inputRdFactory, 6, new StartComputeUpdateFunctionFactory(confFactory),
+ preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -163,18 +155,18 @@
* construct bulk-load index operator
*/
int[] fieldPermutation = new int[] { 0, 1 };
+ int[] keyFields = new int[] { 0 };
IBinaryComparatorFactory[] indexCmpFactories = new IBinaryComparatorFactory[1];
indexCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration + 1,
WritableComparator.get(vertexIdClass).getClass());
TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
storageManagerInterface, lcManagerProvider, secondaryFileSplitProvider, typeTraits, indexCmpFactories,
- fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
+ fieldPermutation, keyFields, DEFAULT_BTREE_FILL_FACTOR, getIndexDataflowHelperFactory());
ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
/**
* construct local sort operator
*/
- int[] keyFields = new int[] { 0 };
INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
@@ -223,7 +215,7 @@
*/
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
@@ -234,7 +226,7 @@
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -245,8 +237,7 @@
EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
ClusterConfig.setLocationConstraint(spec, emptySink4);
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
/** connect all operators **/
@@ -339,14 +330,9 @@
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
typeTraits[1] = new TypeTraits(false);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
IndexNestedLoopJoinOperatorDescriptor setUnion = new IndexNestedLoopJoinOperatorDescriptor(spec, rdFinal,
- storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderRead, interiorFrameFactory,
- leafFrameFactory, typeTraits, comparatorFactories, true, keyFields, keyFields, true, true,
- new BTreeDataflowHelperFactory(), true);
+ storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderRead, typeTraits,
+ comparatorFactories, true, keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true);
ClusterConfig.setLocationConstraint(spec, setUnion);
/**
@@ -364,7 +350,7 @@
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
+ getIndexDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
ClusterConfig.setLocationConstraint(spec, join);
@@ -379,7 +365,8 @@
IFileSplitProvider secondaryFileSplitProviderWrite = ClusterConfig.getFileSplitProvider(jobId, writeFile);
TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderWrite, typeTraits,
- indexCmpFactories, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
+ indexCmpFactories, fieldPermutation, keyFields, DEFAULT_BTREE_FILL_FACTOR,
+ getIndexDataflowHelperFactory());
ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
/**
@@ -447,7 +434,7 @@
*/
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
@@ -458,7 +445,7 @@
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -470,8 +457,7 @@
ClusterConfig.setLocationConstraint(spec, emptySink4);
ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index 5faf122..3af8921 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -35,14 +35,9 @@
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -59,9 +54,9 @@
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -72,8 +67,6 @@
import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
public class JobGenOuterJoin extends JobGen {
@@ -129,12 +122,11 @@
vertexClass.getName());
RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
- BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+ TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 5,
- new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete);
+ getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
+ preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -206,8 +198,8 @@
int[] fieldPermutation = new int[] { 0, 1 };
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+ NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
/**
@@ -216,8 +208,8 @@
int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+ null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -227,8 +219,7 @@
EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
ClusterConfig.setLocationConstraint(spec, emptySink4);
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -318,10 +309,6 @@
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
typeTraits[1] = new TypeTraits(false);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
@@ -335,11 +322,10 @@
vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
- spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory,
- leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
- keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
- new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete);
+ spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+ JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true,
+ nullWriterFactories, inputRdFactory, 5, new ComputeUpdateFunctionFactory(confFactory), preHookFactory,
+ null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -408,8 +394,8 @@
int[] fieldPermutation = new int[] { 0, 1 };
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+ NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
/**
@@ -418,7 +404,7 @@
int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
@@ -431,8 +417,7 @@
ClusterConfig.setLocationConstraint(spec, emptySink4);
ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index fee738e..50949aa 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -34,14 +34,9 @@
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -58,9 +53,9 @@
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -71,8 +66,6 @@
import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
public class JobGenOuterJoinSingleSort extends JobGen {
@@ -131,12 +124,11 @@
vertexClass.getName());
RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
- BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+ TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 5,
- new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete);
+ getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
+ preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -198,8 +190,8 @@
int[] fieldPermutation = new int[] { 0, 1 };
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+ NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
/**
@@ -208,8 +200,8 @@
int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+ null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -221,8 +213,7 @@
ClusterConfig.setLocationConstraint(spec, emptySink4);
ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -307,10 +298,6 @@
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
typeTraits[1] = new TypeTraits(false);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
@@ -324,11 +311,10 @@
vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
- spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory,
- leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
- keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
- new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete);
+ spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+ JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true,
+ nullWriterFactories, inputRdFactory, 5, new ComputeUpdateFunctionFactory(confFactory), preHookFactory,
+ null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -385,8 +371,8 @@
int[] fieldPermutation = new int[] { 0, 1 };
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+ NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
/**
@@ -395,8 +381,8 @@
int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+ null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -408,8 +394,7 @@
ClusterConfig.setLocationConstraint(spec, emptySink4);
ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index eef6b7e..362e413 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -34,14 +34,9 @@
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -58,9 +53,9 @@
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -71,8 +66,6 @@
import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
public class JobGenOuterJoinSort extends JobGen {
@@ -128,12 +121,11 @@
vertexClass.getName());
RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
- BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+ TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 5,
- new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete);
+ getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
+ preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -212,8 +204,8 @@
int[] fieldPermutation = new int[] { 0, 1 };
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+ NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
/**
@@ -222,8 +214,8 @@
int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+ null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -234,8 +226,7 @@
EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
ClusterConfig.setLocationConstraint(spec, emptySink4);
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -321,10 +312,6 @@
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
typeTraits[1] = new TypeTraits(false);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
- typeTraits));
INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
@@ -338,11 +325,10 @@
vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
- spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory,
- leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
- keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
- new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdInsert, rdDelete);
+ spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+ JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true,
+ nullWriterFactories, inputRdFactory, 5, new ComputeUpdateFunctionFactory(confFactory), preHookFactory,
+ null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -418,8 +404,8 @@
int[] fieldPermutation = new int[] { 0, 1 };
TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+ NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, insertOp);
/**
@@ -428,8 +414,8 @@
int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
- new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+ null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -441,8 +427,7 @@
ClusterConfig.setLocationConstraint(spec, emptySink4);
ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
- ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
- new WritableSerializerDeserializerFactory(vertexIdClass));
+ ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index 56fd676..73b053f 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -67,6 +67,7 @@
ncConfig1.dataIPAddress = "127.0.0.1";
ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
+ ncConfig1.ioDevices="dev1,dev2";
ncConfig1.appNCMainClass = NCApplicationEntryPoint.class.getName();
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -79,6 +80,7 @@
ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
ncConfig2.appNCMainClass = NCApplicationEntryPoint.class.getName();
+ ncConfig2.ioDevices="dev1,dev2";
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index 24a96a3..4c7f91d 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -278,7 +278,7 @@
typeTraits[i] = new TypeTraits(false);
TreeIndexBulkLoadOperatorDescriptor writer = new TreeIndexBulkLoadOperatorDescriptor(spec,
storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
- fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, new BTreeDataflowHelperFactory(),
+ fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, false, new BTreeDataflowHelperFactory(),
NoOpOperationCallbackFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, NC1_ID, NC2_ID);
diff --git a/pregelix/pregelix-dataflow-std/pom.xml b/pregelix/pregelix-dataflow-std/pom.xml
index 0a478d4..3604e57 100644
--- a/pregelix/pregelix-dataflow-std/pom.xml
+++ b/pregelix/pregelix-dataflow-std/pom.xml
@@ -143,6 +143,13 @@
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-btree</artifactId>
+ <version>0.2.7-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-control-cc</artifactId>
<version>0.2.7-SNAPSHOT</version>
<type>jar</type>
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
index 0b3a7fe..3403afc 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
@@ -23,10 +23,10 @@
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -91,8 +91,7 @@
public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
- IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
boolean isRightOuter, INullWriterFactory[] nullWriterFactories, IRecordDescriptorFactory inputRdFactory,
@@ -125,8 +124,7 @@
public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
- IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
boolean isSetUnion, IRecordDescriptorFactory inputRdFactory, int outputArity,
@@ -158,7 +156,7 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
if (isRightOuter) {
INullWriter[] nullWriters = new INullWriter[nullWriterFactories.length];
for (int i = 0; i < nullWriters.length; i++)
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
index bd45509..5156dbf 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
@@ -28,15 +28,12 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -49,12 +46,12 @@
import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
- private TreeIndexDataflowHelper treeIndexOpHelper;
+ private IndexDataflowHelper treeIndexOpHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
private FrameTupleAppender appender;
- private BTree btree;
+ private ITreeIndex index;
private PermutingFrameTupleReference lowKey;
private PermutingFrameTupleReference highKey;
private boolean lowKeyInclusive;
@@ -62,9 +59,8 @@
private RangePredicate rangePred;
private MultiComparator lowKeySearchCmp;
private MultiComparator highKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- protected ITreeIndexAccessor indexAccessor;
+ private IIndexCursor cursor;
+ protected IIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
private final IFrameWriter[] writers;
@@ -77,8 +73,9 @@
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
- IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity) {
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity)
+ throws HyracksDataException {
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
@@ -99,7 +96,7 @@
}
protected void setCursor() {
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+ cursor = indexAccessor.createSearchCursor();
}
@Override
@@ -112,13 +109,11 @@
try {
treeIndexOpHelper.open();
- btree = (BTree) treeIndexOpHelper.getIndexInstance();
- cursorFrame = btree.getLeafFrameFactory().createFrame();
- setCursor();
+ index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
// TODO: Can we construct the multicmps using helper methods?
- int lowKeySearchFields = btree.getComparatorFactories().length;
- int highKeySearchFields = btree.getComparatorFactories().length;
+ int lowKeySearchFields = index.getComparatorFactories().length;
+ int highKeySearchFields = index.getComparatorFactories().length;
if (lowKey != null)
lowKeySearchFields = lowKey.getFieldCount();
if (highKey != null)
@@ -126,7 +121,7 @@
IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
@@ -135,10 +130,9 @@
} else {
IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
for (int i = 0; i < highKeySearchFields; i++) {
- highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
highKeySearchCmp = new MultiComparator(highKeySearchComparators);
-
}
rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
@@ -147,9 +141,10 @@
appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
- updateBuffer.setFieldCount(btree.getFieldCount());
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ setCursor();
+ cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+ updateBuffer.setFieldCount(index.getFieldCount());
} catch (Exception e) {
treeIndexOpHelper.close();
throw new HyracksDataException(e);
@@ -201,7 +196,7 @@
try {
cursor.close();
//batch update
- updateBuffer.updateBTree(indexAccessor);
+ updateBuffer.updateIndex(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
@@ -226,4 +221,4 @@
writers[index] = writer;
}
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
index 6dc713c..440ae86 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
@@ -87,8 +87,7 @@
public IndexNestedLoopJoinOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
- IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
boolean isSetUnion) {
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
index d3114d2..2ddca90 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
@@ -29,18 +29,18 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
public class IndexNestedLoopJoinOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private TreeIndexDataflowHelper treeIndexOpHelper;
+ private IndexDataflowHelper treeIndexOpHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
@@ -48,7 +48,7 @@
private ArrayTupleBuilder tb;
private DataOutput dos;
- private BTree btree;
+ private ITreeIndex index;
private PermutingFrameTupleReference lowKey;
private PermutingFrameTupleReference highKey;
private boolean lowKeyInclusive;
@@ -57,7 +57,7 @@
private MultiComparator lowKeySearchCmp;
private MultiComparator highKeySearchCmp;
private IIndexCursor cursor;
- protected ITreeIndexAccessor indexAccessor;
+ protected IIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
private final RecordDescriptor inputRecDesc;
@@ -66,7 +66,7 @@
int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
@@ -91,11 +91,11 @@
try {
treeIndexOpHelper.open();
- btree = (BTree) treeIndexOpHelper.getIndexInstance();
+ index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
writer.open();
- int lowKeySearchFields = btree.getComparatorFactories().length;
- int highKeySearchFields = btree.getComparatorFactories().length;
+ int lowKeySearchFields = index.getComparatorFactories().length;
+ int highKeySearchFields = index.getComparatorFactories().length;
if (lowKey != null)
lowKeySearchFields = lowKey.getFieldCount();
if (highKey != null)
@@ -103,7 +103,7 @@
IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
@@ -112,7 +112,7 @@
} else {
IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
for (int i = 0; i < highKeySearchFields; i++) {
- highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
highKeySearchCmp = new MultiComparator(highKeySearchComparators);
}
@@ -120,11 +120,11 @@
rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
highKeySearchCmp);
writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
+ tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + index.getFieldCount());
dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
setCursor();
} catch (Exception e) {
treeIndexOpHelper.close();
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
index 289128e..4ca7533 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
@@ -30,15 +30,12 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -52,7 +49,7 @@
public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends
AbstractUnaryInputOperatorNodePushable {
- private TreeIndexDataflowHelper treeIndexOpHelper;
+ private IndexDataflowHelper treeIndexOpHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
@@ -60,13 +57,12 @@
private ArrayTupleBuilder nullTupleBuilder;
private DataOutput dos;
- private BTree btree;
+ private ITreeIndex index;
private RangePredicate rangePred;
private MultiComparator lowKeySearchCmp;
private MultiComparator highKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- protected ITreeIndexAccessor indexAccessor;
+ private IIndexCursor cursor;
+ protected IIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
private final RecordDescriptor inputRecDesc;
@@ -88,9 +84,9 @@
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, INullWriter[] nullWriter, IUpdateFunctionFactory functionFactory,
IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
- IRecordDescriptorFactory inputRdFactory, int outputArity) {
+ IRecordDescriptorFactory inputRdFactory, int outputArity) throws HyracksDataException {
inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -111,7 +107,7 @@
}
protected void setCursor() {
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+ cursor = indexAccessor.createSearchCursor();
}
@Override
@@ -124,18 +120,16 @@
try {
treeIndexOpHelper.open();
- btree = (BTree) treeIndexOpHelper.getIndexInstance();
- cursorFrame = btree.getLeafFrameFactory().createFrame();
- setCursor();
+ index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
// construct range predicate
// TODO: Can we construct the multicmps using helper methods?
- int lowKeySearchFields = btree.getComparatorFactories().length;
- int highKeySearchFields = btree.getComparatorFactories().length;
+ int lowKeySearchFields = index.getComparatorFactories().length;
+ int highKeySearchFields = index.getComparatorFactories().length;
IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
@@ -144,7 +138,7 @@
} else {
IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
for (int i = 0; i < highKeySearchFields; i++) {
- highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
highKeySearchCmp = new MultiComparator(highKeySearchComparators);
}
@@ -164,7 +158,8 @@
appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ setCursor();
/** set the search cursor */
rangePred.setLowKey(null, true);
@@ -179,8 +174,8 @@
match = false;
}
- cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
- updateBuffer.setFieldCount(btree.getFieldCount());
+ cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+ updateBuffer.setFieldCount(index.getFieldCount());
} catch (Exception e) {
treeIndexOpHelper.close();
throw new HyracksDataException(e);
@@ -239,7 +234,7 @@
try {
cursor.close();
//batch update
- updateBuffer.updateBTree(indexAccessor);
+ updateBuffer.updateIndex(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
index 6f7643e..dd6ee3c 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
@@ -30,22 +30,19 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
public class IndexNestedLoopRightOuterJoinOperatorNodePushable extends
AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private TreeIndexDataflowHelper treeIndexOpHelper;
+ private IndexDataflowHelper treeIndexOpHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
@@ -53,14 +50,13 @@
private ArrayTupleBuilder tb;
private DataOutput dos;
- private BTree btree;
+ private ITreeIndex index;
private boolean isForward;
private RangePredicate rangePred;
private MultiComparator lowKeySearchCmp;
private MultiComparator highKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- protected ITreeIndexAccessor indexAccessor;
+ private IIndexCursor cursor;
+ protected IIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
private final RecordDescriptor inputRecDesc;
@@ -76,7 +72,7 @@
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, INullWriter[] nullWriter) {
inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.isForward = isForward;
this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -93,7 +89,7 @@
}
protected void setCursor() {
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+ cursor = indexAccessor.createSearchCursor();
}
@Override
@@ -101,19 +97,17 @@
accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
try {
treeIndexOpHelper.open();
- btree = (BTree) treeIndexOpHelper.getIndexInstance();
- cursorFrame = btree.getLeafFrameFactory().createFrame();
- setCursor();
+ index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
writer.open();
// construct range predicate
// TODO: Can we construct the multicmps using helper methods?
- int lowKeySearchFields = btree.getComparatorFactories().length;
- int highKeySearchFields = btree.getComparatorFactories().length;
+ int lowKeySearchFields = index.getComparatorFactories().length;
+ int highKeySearchFields = index.getComparatorFactories().length;
IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
@@ -122,7 +116,7 @@
} else {
IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
for (int i = 0; i < highKeySearchFields; i++) {
- highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
highKeySearchCmp = new MultiComparator(highKeySearchComparators);
@@ -131,12 +125,13 @@
rangePred = new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp);
writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
+ tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + index.getFieldCount());
dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ setCursor();
/** set the search cursor */
rangePred.setLowKey(null, true);
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
index c978614..3cebfb8 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
@@ -28,15 +28,12 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -49,19 +46,18 @@
import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
- private TreeIndexDataflowHelper treeIndexOpHelper;
+ private IndexDataflowHelper treeIndexOpHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
private FrameTupleAppender appender;
- private BTree btree;
+ private ITreeIndex index;
private boolean isForward;
private RangePredicate rangePred;
private MultiComparator lowKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- protected ITreeIndexAccessor indexAccessor;
+ private IIndexCursor cursor;
+ protected IIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
private PermutingFrameTupleReference lowKey;
@@ -80,8 +76,8 @@
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, IUpdateFunctionFactory functionFactory,
IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
- IRecordDescriptorFactory inputRdFactory, int outputArity) {
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ IRecordDescriptorFactory inputRdFactory, int outputArity) throws HyracksDataException {
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.isForward = isForward;
this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -102,7 +98,7 @@
}
protected void setCursor() {
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+ cursor = indexAccessor.createSearchCursor();
}
@Override
@@ -112,15 +108,13 @@
try {
treeIndexOpHelper.open();
- btree = (BTree) treeIndexOpHelper.getIndexInstance();
- cursorFrame = btree.getLeafFrameFactory().createFrame();
- setCursor();
+ index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
rangePred = new RangePredicate(null, null, true, true, null, null);
- int lowKeySearchFields = btree.getComparatorFactories().length;
+ int lowKeySearchFields = index.getComparatorFactories().length;
IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
@@ -128,7 +122,8 @@
appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ setCursor();
/** set the search cursor */
rangePred.setLowKey(null, true);
@@ -142,8 +137,8 @@
currentTopTuple = cursor.getTuple();
match = false;
}
- cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
- updateBuffer.setFieldCount(btree.getFieldCount());
+ cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+ updateBuffer.setFieldCount(index.getFieldCount());
} catch (Exception e) {
treeIndexOpHelper.close();
throw new HyracksDataException(e);
@@ -210,7 +205,7 @@
cursor.close();
//batch update
- updateBuffer.updateBTree(indexAccessor);
+ updateBuffer.updateIndex(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
@@ -256,4 +251,4 @@
public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
writers[index] = writer;
}
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
index eef7c85..bbe2764 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
@@ -29,21 +29,18 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
public class IndexNestedLoopSetUnionOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private TreeIndexDataflowHelper treeIndexOpHelper;
+ private IndexDataflowHelper treeIndexOpHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
@@ -51,12 +48,11 @@
private ArrayTupleBuilder tb;
private DataOutput dos;
- private BTree btree;
+ private ITreeIndex index;
private RangePredicate rangePred;
private MultiComparator lowKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- protected ITreeIndexAccessor indexAccessor;
+ private IIndexCursor cursor;
+ protected IIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
private final RecordDescriptor inputRecDesc;
@@ -71,7 +67,7 @@
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields) {
inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -86,7 +82,7 @@
}
protected void setCursor() {
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+ cursor = indexAccessor.createSearchCursor();
}
@Override
@@ -95,26 +91,25 @@
try {
treeIndexOpHelper.open();
- btree = (BTree) treeIndexOpHelper.getIndexInstance();
- cursorFrame = btree.getLeafFrameFactory().createFrame();
- setCursor();
+ index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
writer.open();
rangePred = new RangePredicate(null, null, true, true, null, null);
- int lowKeySearchFields = btree.getComparatorFactories().length;
+ int lowKeySearchFields = index.getComparatorFactories().length;
IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
}
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(btree.getFieldCount());
+ tb = new ArrayTupleBuilder(index.getFieldCount());
dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ setCursor();
/** set the search cursor */
rangePred.setLowKey(null, true);
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
index 1dc1529..4dd78c3 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
@@ -42,9 +42,9 @@
public TreeIndexBulkReLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
IIndexLifecycleManagerProvider lcManagerProvider, IFileSplitProvider fileSplitProvider,
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
- float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
+ int[] bloomFilterFields, float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
super(spec, 1, 0, null, storageManager, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
- fieldPermutation, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+ bloomFilterFields, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
this.fieldPermutation = fieldPermutation;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
index 208d41c..c4890e1 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
@@ -29,13 +29,13 @@
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public class TreeIndexBulkReLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
private final float fillFactor;
- private final TreeIndexDataflowHelper treeIndexOpHelper;
+ private final IndexDataflowHelper treeIndexOpHelper;
private final IIndexOperatorDescriptor opDesc;
private final IRecordDescriptorProvider recordDescProvider;
private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
@@ -49,7 +49,7 @@
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
IFileSplitProvider fileSplitProvider) {
this.fillFactor = fillFactor;
- treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.opDesc = opDesc;
this.recordDescProvider = recordDescProvider;
@@ -60,11 +60,12 @@
public void open() throws HyracksDataException {
RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
+ treeIndexOpHelper.destroy();
treeIndexOpHelper.create();
treeIndexOpHelper.open();
try {
index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
- bulkLoader = index.createBulkLoader(fillFactor, false, 0);
+ bulkLoader = index.createBulkLoader(fillFactor, false, 0, false);
} catch (Exception e) {
// cleanup in case of failure
treeIndexOpHelper.close();
@@ -99,5 +100,6 @@
@Override
public void fail() throws HyracksDataException {
+
}
}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
similarity index 90%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorDescriptor.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
index 784288f..31496a9 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
@@ -21,6 +21,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
@@ -33,7 +34,7 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
-public class BTreeSearchFunctionUpdateOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+public class TreeSearchFunctionUpdateOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
private static final long serialVersionUID = 1L;
@@ -52,7 +53,7 @@
private final int outputArity;
- public BTreeSearchFunctionUpdateOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+ public TreeSearchFunctionUpdateOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
@@ -83,9 +84,9 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
- return new BTreeSearchFunctionUpdateOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ return new TreeSearchFunctionUpdateOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward,
lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, functionFactory, preHookFactory,
postHookFactory, inputRdFactory, outputArity);
}
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
similarity index 73%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
index 0216b52..bd85e3e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
@@ -19,6 +19,7 @@
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -27,16 +28,13 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -48,8 +46,8 @@
import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference;
import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
-public class BTreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
- protected TreeIndexDataflowHelper treeIndexHelper;
+public class TreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
+ protected IndexDataflowHelper treeIndexHelper;
protected FrameTupleAccessor accessor;
protected ByteBuffer writeBuffer;
@@ -57,7 +55,7 @@
protected ArrayTupleBuilder tb;
protected DataOutput dos;
- protected BTree btree;
+ protected ITreeIndex index;
protected boolean isForward;
protected PermutingFrameTupleReference lowKey;
protected PermutingFrameTupleReference highKey;
@@ -66,9 +64,11 @@
protected RangePredicate rangePred;
protected MultiComparator lowKeySearchCmp;
protected MultiComparator highKeySearchCmp;
- protected ITreeIndexCursor cursor;
+ protected IIndexCursor cursor;
protected ITreeIndexFrame cursorFrame;
- protected ITreeIndexAccessor indexAccessor;
+ protected IIndexAccessor indexAccessor;
+ protected int[] lowKeyFields;
+ protected int[] highKeyFields;
protected RecordDescriptor recDesc;
@@ -78,16 +78,19 @@
private final UpdateBuffer updateBuffer;
private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference();
- public BTreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
+ public TreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
- IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity) {
- treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity)
+ throws HyracksDataException {
+ treeIndexHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.isForward = isForward;
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
+ this.lowKeyFields = lowKeyFields;
+ this.highKeyFields = highKeyFields;
this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
if (lowKeyFields != null && lowKeyFields.length > 0) {
lowKey = new PermutingFrameTupleReference();
@@ -114,25 +117,46 @@
try {
treeIndexHelper.open();
- btree = (BTree) treeIndexHelper.getIndexInstance();
- cursorFrame = btree.getLeafFrameFactory().createFrame();
- setCursor();
+ index = (ITreeIndex) treeIndexHelper.getIndexInstance();
+ cursorFrame = index.getLeafFrameFactory().createFrame();
// Construct range predicate.
- lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getComparatorFactories(), lowKey);
- highKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getComparatorFactories(), highKey);
+ int lowKeySearchFields = index.getComparatorFactories().length;
+ int highKeySearchFields = index.getComparatorFactories().length;
+ if (lowKey != null)
+ lowKeySearchFields = lowKey.getFieldCount();
+ if (highKey != null)
+ highKeySearchFields = highKey.getFieldCount();
+
+ IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
+ for (int i = 0; i < lowKeySearchFields; i++) {
+ lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
+ }
+ lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
+
+ if (lowKeySearchFields == highKeySearchFields) {
+ highKeySearchCmp = lowKeySearchCmp;
+ } else {
+ IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
+ for (int i = 0; i < highKeySearchFields; i++) {
+ highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
+ }
+ highKeySearchCmp = new MultiComparator(highKeySearchComparators);
+ }
+
rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
highKeySearchCmp);
writeBuffer = treeIndexHelper.getTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(btree.getFieldCount());
+ tb = new ArrayTupleBuilder(index.getFieldCount());
dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexHelper.getTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ setCursor();
- cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
- updateBuffer.setFieldCount(btree.getFieldCount());
+ cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+ updateBuffer.setFieldCount(index.getFieldCount());
} catch (Exception e) {
treeIndexHelper.close();
throw new HyracksDataException(e);
@@ -140,7 +164,7 @@
}
protected void setCursor() {
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+ cursor = indexAccessor.createSearchCursor();
}
protected void writeSearchResults() throws Exception {
@@ -184,7 +208,7 @@
try {
cursor.close();
//batch update
- updateBuffer.updateBTree(indexAccessor);
+ updateBuffer.updateIndex(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
@@ -209,4 +233,4 @@
writers[index] = writer;
}
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
index b141eaf..8709301 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
@@ -19,15 +19,15 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
public class CopyUpdateUtil {
public static void copyUpdate(SearchKeyTupleReference tempTupleReference, ITupleReference frameTuple,
- UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, ITreeIndexAccessor indexAccessor,
- ITreeIndexCursor cursor, RangePredicate rangePred) throws HyracksDataException, IndexException {
+ UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, IIndexAccessor indexAccessor,
+ IIndexCursor cursor, RangePredicate rangePred) throws HyracksDataException, IndexException {
if (cloneUpdateTb.getSize() > 0) {
int[] fieldEndOffsets = cloneUpdateTb.getFieldEndOffsets();
int srcStart = fieldEndOffsets[0];
@@ -46,7 +46,7 @@
//release the cursor/latch
cursor.close();
//batch update
- updateBuffer.updateBTree(indexAccessor);
+ updateBuffer.updateIndex(indexAccessor);
//try append the to-be-updated tuple again
if (!updateBuffer.appendTuple(cloneUpdateTb)) {
throw new HyracksDataException("cannot append tuple builder!");
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
index 327178c..b2be366 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
@@ -25,7 +25,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
/**
@@ -44,7 +44,7 @@
private final int frameSize;
private IFrameTupleAccessor fta;
- public UpdateBuffer(int numPages, IHyracksTaskContext ctx, int fieldCount) {
+ public UpdateBuffer(int numPages, IHyracksTaskContext ctx, int fieldCount) throws HyracksDataException {
this.appender = new FrameTupleAppender(ctx.getFrameSize());
ByteBuffer buffer = ctx.allocateFrame();
this.buffers.add(buffer);
@@ -55,7 +55,7 @@
this.fta = new UpdateBufferTupleAccessor(frameSize, fieldCount);
}
- public UpdateBuffer(IHyracksTaskContext ctx, int fieldCount) {
+ public UpdateBuffer(IHyracksTaskContext ctx, int fieldCount) throws HyracksDataException {
//by default, the update buffer has 1000 pages
this(1000, ctx, fieldCount);
}
@@ -87,7 +87,7 @@
}
}
- public void updateBTree(ITreeIndexAccessor bta) throws HyracksDataException, IndexException {
+ public void updateIndex(IIndexAccessor bta) throws HyracksDataException, IndexException {
// batch update
for (int i = 0; i <= currentInUse; i++) {
ByteBuffer buffer = buffers.get(i);
@@ -104,7 +104,7 @@
appender.reset(buffer, true);
}
- private void allocate(int index) {
+ private void allocate(int index) throws HyracksDataException {
if (index >= buffers.size()) {
buffers.add(ctx.allocateFrame());
}
diff --git a/pregelix/pregelix-dataflow/pom.xml b/pregelix/pregelix-dataflow/pom.xml
index 962c9f6..2828451 100644
--- a/pregelix/pregelix-dataflow/pom.xml
+++ b/pregelix/pregelix-dataflow/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>pregelix-dataflow</artifactId>
<packaging>jar</packaging>
@@ -134,6 +130,13 @@
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-common</artifactId>
+ <version>0.2.7-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-control-cc</artifactId>
<version>0.2.7-SNAPSHOT</version>
<type>jar</type>
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
index 86da3bf..ca8f190 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
@@ -38,7 +38,8 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
private ByteBuffer frame = ctx.allocateFrame();
private boolean complete = false;
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
index dc9698b..0f41568 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
@@ -25,4 +25,6 @@
public Configuration createConfiguration(IHyracksTaskContext ctx) throws HyracksDataException;
+ public Configuration createConfiguration() throws HyracksDataException;
+
}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/NoBudgetIndexLifecycleManager.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/NoBudgetIndexLifecycleManager.java
new file mode 100644
index 0000000..5484f78
--- /dev/null
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/NoBudgetIndexLifecycleManager.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.dataflow.context;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+
+public class NoBudgetIndexLifecycleManager implements IIndexLifecycleManager, ILifeCycleComponent {
+
+ private final Map<Long, IndexInfo> indexInfos;
+
+ public NoBudgetIndexLifecycleManager() {
+ this.indexInfos = new HashMap<Long, IndexInfo>();
+ }
+
+ @Override
+ public IIndex getIndex(long resourceID) {
+ IndexInfo info = indexInfos.get(resourceID);
+ return info == null ? null : info.index;
+ }
+
+ @Override
+ public void register(long resourceID, IIndex index) throws HyracksDataException {
+ if (indexInfos.containsKey(resourceID)) {
+ throw new HyracksDataException("Index with resource ID " + resourceID + " already exists.");
+ }
+
+ indexInfos.put(resourceID, new IndexInfo(index));
+ }
+
+ @Override
+ public void unregister(long resourceID) throws HyracksDataException {
+ IndexInfo info = indexInfos.remove(resourceID);
+ if (info == null) {
+ throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
+ }
+
+ if (info.referenceCount != 0) {
+ indexInfos.put(resourceID, info);
+ throw new HyracksDataException("Cannot remove index while it is open.");
+ }
+
+ if (info.isOpen) {
+ info.index.deactivate();
+ }
+ }
+
+ @Override
+ public void open(long resourceID) throws HyracksDataException {
+ IndexInfo info = indexInfos.get(resourceID);
+ if (info == null) {
+ throw new HyracksDataException("Failed to open index with resource ID " + resourceID
+ + " since it does not exist.");
+ }
+
+ if (!info.isOpen) {
+ info.index.activate();
+ info.isOpen = true;
+ }
+ info.touch();
+ }
+
+ @Override
+ public void close(long resourceID) {
+ indexInfos.get(resourceID).untouch();
+ }
+
+ private class IndexInfo implements Comparable<IndexInfo> {
+ private final IIndex index;
+ private int referenceCount;
+ private long lastAccess;
+ private boolean isOpen;
+
+ public IndexInfo(IIndex index) {
+ this.index = index;
+ this.lastAccess = -1;
+ this.referenceCount = 0;
+ this.isOpen = false;
+ }
+
+ public void touch() {
+ lastAccess = System.currentTimeMillis();
+ referenceCount++;
+ }
+
+ public void untouch() {
+ lastAccess = System.currentTimeMillis();
+ referenceCount--;
+ }
+
+ @Override
+ public int compareTo(IndexInfo i) {
+ // sort by (isOpen, referenceCount, lastAccess) ascending, where
+ // true < false
+ //
+ // Example sort order:
+ // -------------------
+ // (F, 0, 70) <-- largest
+ // (F, 0, 60)
+ // (T, 10, 80)
+ // (T, 10, 70)
+ // (T, 9, 90)
+ // (T, 0, 100) <-- smallest
+ if (isOpen && !i.isOpen) {
+ return -1;
+ } else if (!isOpen && i.isOpen) {
+ return 1;
+ } else {
+ if (referenceCount < i.referenceCount) {
+ return -1;
+ } else if (referenceCount > i.referenceCount) {
+ return 1;
+ } else {
+ if (lastAccess < i.lastAccess) {
+ return -1;
+ } else if (lastAccess > i.lastAccess) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+ }
+
+ }
+
+ public String toString() {
+ return "{index: " + index + ", isOpen: " + isOpen + ", refCount: " + referenceCount + ", lastAccess: "
+ + lastAccess + "}";
+ }
+ }
+
+ @Override
+ public List<IIndex> getOpenIndexes() {
+ List<IIndex> openIndexes = new ArrayList<IIndex>();
+ for (IndexInfo i : indexInfos.values()) {
+ if (i.isOpen) {
+ openIndexes.add(i.index);
+ }
+ }
+ return openIndexes;
+ }
+
+ @Override
+ public void start() {
+ }
+
+ @Override
+ public void stop(boolean dumpState, OutputStream outputStream) throws IOException {
+ if (dumpState) {
+ dumpState(outputStream);
+ }
+
+ for (IndexInfo i : indexInfos.values()) {
+ if (i.isOpen) {
+ i.index.deactivate();
+ }
+ }
+ }
+
+ private void dumpState(OutputStream os) throws IOException {
+ StringBuilder sb = new StringBuilder();
+
+ String headerFormat = "%-20s %-10s %-20s %-20s %-20s\n";
+ String rowFormat = "%-20d %-10b %-20d %-20s %-20s\n";
+ sb.append(String.format(headerFormat, "ResourceID", "Open", "Reference Count", "Last Access", "Index Name"));
+ IndexInfo ii;
+ for (Map.Entry<Long, IndexInfo> entry : indexInfos.entrySet()) {
+ ii = entry.getValue();
+ sb.append(String.format(rowFormat, entry.getKey(), ii.isOpen, ii.referenceCount, ii.lastAccess, ii.index));
+ }
+ os.write(sb.toString().getBytes());
+ }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index b86691c..24a0a9e 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -30,7 +30,9 @@
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -52,6 +54,7 @@
private final ILocalResourceRepository localResourceRepository;
private final ResourceIdFactory resourceIdFactory;
private final IBufferCache bufferCache;
+ private final IVirtualBufferCache vBufferCache;
private final IFileMapManager fileMapManager;
private final Map<StateKey, IStateObject> appStateMap = new ConcurrentHashMap<StateKey, IStateObject>();
private final Map<String, Long> giraphJobIdToSuperStep = new ConcurrentHashMap<String, Long>();
@@ -76,8 +79,11 @@
bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs,
new PreDelayPageCleanerPolicy(Long.MAX_VALUE), fileMapManager, pageSize, numPages, 1000000,
threadFactory);
+ int numPagesInMemComponents = numPages / 4;
+ vBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(), pageSize,
+ numPagesInMemComponents));
ioManager = (IOManager) appCtx.getRootContext().getIOManager();
- lcManager = new IndexLifecycleManager();
+ lcManager = new NoBudgetIndexLifecycleManager();
localResourceRepository = new TransientLocalResourceRepository();
resourceIdFactory = new ResourceIdFactory(0);
}
@@ -110,6 +116,10 @@
return bufferCache;
}
+ public IVirtualBufferCache getVirtualBufferCache() {
+ return vBufferCache;
+ }
+
public IFileMapProvider getFileMapManager() {
return fileMapManager;
}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index 69c5612..15117a1 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
import edu.uci.ics.pregelix.example.ConnectedComponentsVertex;
import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
import edu.uci.ics.pregelix.example.GraphMutationVertex;
@@ -80,6 +81,7 @@
job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -132,6 +134,7 @@
job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -230,11 +233,12 @@
job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ job.setMutationOrVariableSizedUpdateHeavy(true);
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
}
-
+
private static void generateMaximalCliqueJob2(String jobName, String outputPath) throws IOException {
PregelixJob job = new PregelixJob(jobName);
job.setVertexClass(MaximalCliqueVertex.class);
@@ -243,11 +247,12 @@
job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ job.setMutationOrVariableSizedUpdateHeavy(true);
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH4);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
}
-
+
private static void generateMaximalCliqueJob3(String jobName, String outputPath) throws IOException {
PregelixJob job = new PregelixJob(jobName);
job.setVertexClass(MaximalCliqueVertex.class);
@@ -256,6 +261,8 @@
job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+ job.setMutationOrVariableSizedUpdateHeavy(true);
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH5);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 7b0c7da..f077053 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -94,7 +94,7 @@
@Test
public void test() throws Exception {
setUp();
- Plan[] plans = new Plan[] { Plan.OUTER_JOIN_SORT, Plan.OUTER_JOIN, Plan.INNER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT };
+ Plan[] plans = new Plan[] { Plan.INNER_JOIN, Plan.OUTER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT, Plan.OUTER_JOIN_SORT };
for (Plan plan : plans) {
driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
index 199efab..decbde8 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
index 46444b3..cca66bb 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -137,6 +122,7 @@
<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat</value></property>
<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
<property><name>hadoop.logfile.size</name><value>10000000</value></property>
<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
index 47db410..d5ec8f1 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
index c08853b..b4c42e6 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -95,6 +80,7 @@
<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>pregelix.updateIntensive</name><value>true</value></property>
<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
<property><name>mapred.queue.names</name><value>default</value></property>
<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
index aafabee..6cf075b 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -95,6 +80,7 @@
<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>pregelix.updateIntensive</name><value>true</value></property>
<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
<property><name>mapred.queue.names</name><value>default</value></property>
<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
index ee13335..49e2e6f 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -95,6 +80,7 @@
<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>pregelix.updateIntensive</name><value>true</value></property>
<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
<property><name>mapred.queue.names</name><value>default</value></property>
<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
@@ -135,6 +121,7 @@
<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex$MaximalCliqueVertexOutputFormat</value></property>
<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
<property><name>hadoop.logfile.size</name><value>10000000</value></property>
<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
index 5f03027..65e0b30 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
index 9ab92ff..9e1e0b0 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
index 314ca55..c4366d7 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -137,6 +122,7 @@
<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat</value></property>
<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
<property><name>hadoop.logfile.size</name><value>10000000</value></property>
<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
index f8a2055..c05a4da 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
index f1c72fa..ac0d508 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
index 427c0ea..225429a 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
index 7e96163..bd9da92 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
index 823e8b0..9acd7bc 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
index 4ae102b..6c25575 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
index 228a526..4a40a6a 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index ae9f47e..54e2256 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -148,6 +148,13 @@
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-common</artifactId>
+ <version>0.2.7-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-control-cc</artifactId>
<version>0.2.7-SNAPSHOT</version>
<type>jar</type>
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
new file mode 100644
index 0000000..ec51047
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.runtime.bootstrap;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
+
+/**
+ * The virtual buffer cache provider
+ *
+ * @author yingyib
+ */
+public class VirtualBufferCacheProvider implements IVirtualBufferCacheProvider {
+
+ private static final long serialVersionUID = 1L;
+
+ public VirtualBufferCacheProvider(){
+
+ }
+
+ @Override
+ public synchronized IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+ return RuntimeContext.get(ctx).getVirtualBufferCache();
+ }
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
new file mode 100644
index 0000000..88577c2
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.runtime.function;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction;
+import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
+
+/**
+ * No operation update function factory
+ *
+ * @author yingyib
+ */
+public class NoOpUpdateFunctionFactory implements IUpdateFunctionFactory {
+ private static final long serialVersionUID = 1L;
+ public static NoOpUpdateFunctionFactory INSTANCE = new NoOpUpdateFunctionFactory();
+
+ private NoOpUpdateFunctionFactory() {
+
+ }
+
+ @Override
+ public IUpdateFunction createFunction() {
+ return new IUpdateFunction() {
+
+ @Override
+ public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writer)
+ throws HyracksDataException {
+
+ }
+
+ @Override
+ public void process(Object[] tuple) throws HyracksDataException {
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+
+ }
+
+ };
+ }
+
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexPartitionComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexPartitionComputerFactory.java
new file mode 100644
index 0000000..acccabb
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexPartitionComputerFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.runtime.touchpoint;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+
+/**
+ * The vertex-based partition computer factory.
+ * It is used to support customized graph partitioning function.
+ *
+ * @author yingyib
+ */
+public class VertexPartitionComputerFactory implements ITuplePartitionComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final IConfigurationFactory confFactory;
+
+ public VertexPartitionComputerFactory(IConfigurationFactory confFactory) {
+ this.confFactory = confFactory;
+ }
+
+ @SuppressWarnings("rawtypes")
+ public ITuplePartitionComputer createPartitioner() {
+ try {
+ final Configuration conf = confFactory.createConfiguration();
+ return new ITuplePartitionComputer() {
+ private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+ private final DataInputStream dis = new DataInputStream(bbis);
+ private final VertexPartitioner partitioner = BspUtils.createVertexPartitioner(conf);
+ private final WritableComparable vertexId = BspUtils.createVertexIndex(conf);
+
+ @SuppressWarnings("unchecked")
+ public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ try {
+ int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+ + accessor.getFieldStartOffset(tIndex, 0);
+ bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+ vertexId.readFields(dis);
+ return Math.abs(partitioner.getPartitionId(vertexId, nParts) % nParts);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ };
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+}