Merge branch 'genomix/fullstack_genomix' into wbiesing/refactor_p4
use static final values "debug" and "problemKmers" to log all actions
for given set of kmers
Conflicts:
genomix/genomix-
pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
index 9955a63..3f1aae2 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
@@ -44,9 +44,11 @@
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -142,27 +144,32 @@
int n = orderColumns.size();
int[] sortFields = new int[n];
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[n];
- {
- int j = 0;
- for (OrderColumn oc : orderColumns) {
- LogicalVariable var = oc.getColumn();
- sortFields[j] = opSchema.findVariable(var);
- Object type = env.getVarType(var);
- IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
- comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
- j++;
+
+ INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+ INormalizedKeyComputerFactory nkcf = null;
+
+ int j = 0;
+ for (OrderColumn oc : orderColumns) {
+ LogicalVariable var = oc.getColumn();
+ sortFields[j] = opSchema.findVariable(var);
+ Object type = env.getVarType(var);
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
+ if (j == 0 && nkcfProvider != null && type != null) {
+ nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, oc.getOrder() == OrderKind.ASC);
}
+ j++;
}
IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields,
- comparatorFactories);
+ comparatorFactories, nkcf);
return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
}
-
+
public List<LogicalVariable> getPartitionFields() {
return partitionFields;
}
-
+
public List<OrderColumn> getOrderColumns() {
return orderColumns;
}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
index dc345ac..b03b99d 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
@@ -39,9 +39,11 @@
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -128,6 +130,10 @@
IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
IBinaryHashFunctionFactory[] hashFuns = new IBinaryHashFunctionFactory[n];
IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+
+ INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+ INormalizedKeyComputerFactory nkcf = null;
+
for (int i = 0; i < n; i++) {
sortFields[i] = opSchema.findVariable(sortColumns[i].getColumn());
Object type = env.getVarType(sortColumns[i].getColumn());
@@ -135,9 +141,12 @@
comps[i] = bcfp.getBinaryComparatorFactory(type, sortColumns[i].getOrder() == OrderKind.ASC);
IBinaryHashFunctionFactoryProvider bhffp = context.getBinaryHashFunctionFactoryProvider();
hashFuns[i] = bhffp.getBinaryHashFunctionFactory(type);
+ if (i == 0 && nkcfProvider != null && type != null) {
+ nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, sortColumns[i].getOrder() == OrderKind.ASC);
+ }
}
ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(sortFields, hashFuns);
- IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps);
+ IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.ONE);
}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index b299e78..131eea0 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -24,7 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -52,12 +52,12 @@
return new AbstractOneInputOneOutputPushRuntime() {
- FrameSorter frameSorter = null;
+ FrameSorterMergeSort frameSorter = null;
@Override
public void open() throws HyracksDataException {
if (frameSorter == null) {
- frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+ frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
outputRecordDesc);
}
frameSorter.reset();
diff --git a/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java b/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java
index ffcb449..0250a73 100644
--- a/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java
+++ b/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java
@@ -21,8 +21,10 @@
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.Serializable;
+import java.util.AbstractMap;
import java.util.Comparator;
import java.util.EnumSet;
+import java.util.Map;
import org.apache.hadoop.io.WritableComparable;
@@ -778,6 +780,21 @@
break;
}
}
+
+ /**
+ * Debug helper function to find the edge associated with the given kmer.
+ *
+ * Note: may be very slow-- does a linear scan of all edges!
+ */
+ public Map.Entry<EDGETYPE, EdgeWritable> findEdge(final VKmerBytesWritable kmer) {
+ for (EDGETYPE dir : EDGETYPE.values()) {
+ for (EdgeWritable e : edges[dir.get()]) {
+ if (e.getKey().equals(kmer))
+ return new AbstractMap.SimpleEntry<EDGETYPE, EdgeWritable>(dir, e);
+ }
+ }
+ return null;
+ }
public int inDegree() {
return edges[EDGETYPE.RR.get()].getCountOfPosition() + edges[EDGETYPE.RF.get()].getCountOfPosition();
diff --git a/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java b/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java
index 488bc45..831cc17 100644
--- a/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java
+++ b/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java
@@ -58,6 +58,7 @@
*/
public class GenomixDriver {
+ public static final Logger GENOMIX_ROOT_LOG = Logger.getLogger("edu.uci.ics.genomix"); // here only so we can control children loggers
private static final Logger LOG = Logger.getLogger(GenomixDriver.class.getName());
private static final String HADOOP_CONF = "hadoop.conf.xml";
private String prevOutput;
diff --git a/genomix/genomix-driver/src/main/resources/conf/logging.properties b/genomix/genomix-driver/src/main/resources/conf/logging.properties
index 5269ae1..bdb1318 100644
--- a/genomix/genomix-driver/src/main/resources/conf/logging.properties
+++ b/genomix/genomix-driver/src/main/resources/conf/logging.properties
@@ -20,8 +20,8 @@
# To also add the FileHandler, use the following line instead.
-#handlers= java.util.logging.FileHandler, java.util.logging.ConsoleHandler
-handlers= java.util.logging.FileHandler
+handlers= java.util.logging.FileHandler, java.util.logging.ConsoleHandler
+#handlers= java.util.logging.FileHandler
# Default global logging level.
# This specifies which kinds of events are logged across
@@ -30,8 +30,8 @@
# Note that the ConsoleHandler also has a separate level
# setting to limit messages printed to the console.
-#.level= SEVERE
- .level= INFO
+.level= SEVERE
+# .level= INFO
# .level= FINE
# .level = FINEST
@@ -42,18 +42,18 @@
# default file output is in user's home directory.
- java.util.logging.FileHandler.pattern = logs/genomix-pregelix-tests.log
+ java.util.logging.FileHandler.pattern = genomix-%u.log
java.util.logging.FileHandler.limit = 0
java.util.logging.FileHandler.count = 1
# java.util.logging.FileHandler.formatter = java.util.logging.XMLFormatter
-# java.util.logging.FileHandler.formatter = java.util.logging.SimpleFormatter
- java.util.logging.FileHandler.formatter = edu.uci.ics.genomix.pregelix.log.PathMergeLogFormatter
+ java.util.logging.FileHandler.formatter = java.util.logging.SimpleFormatter
+# java.util.logging.FileHandler.formatter = edu.uci.ics.genomix.pregelix.log.PathMergeLogFormatter
java.util.logging.FileHandler.level = FINE
# Limit the message that are printed on the console to FINE and above.
-#java.util.logging.ConsoleHandler.level = INFO
-#java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter
+java.util.logging.ConsoleHandler.level = INFO
+java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter
#java.util.logging.ConsoleHandler.formatter = java.util.logging.XMLFormatter
@@ -65,7 +65,8 @@
# For example, set the com.xyz.foo logger to only log SEVERE
# messages:
-edu.uci.ics.genomix.pregelix.level = FINE
+edu.uci.ics.genomix.level = ALL
+edu.uci.ics.pregelix.core.driver.Driver.level = INFO
#edu.uci.ics.asterix.level = FINE
#edu.uci.ics.algebricks.level = FINE
#edu.uci.ics.hyracks.level = SEVERE
diff --git a/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py b/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py
index 4e73653..d84b026 100644
--- a/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py
+++ b/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py
@@ -3,24 +3,31 @@
import sys
import string
import pickle
+from itertools import imap
-
-def getneighbors(readids, kmerlength, reads, index, skipreads):
- "return a set of readids containing any of the kmers in the given readid"
+def getneighbors(ids_to_search, kmerlength, readindex, kmerindex, skipids):
+ """return a set of ids_to_search containing any of the kmers in the given readid
+ :param dict ids_to_search:
+ :param int kmerlength:
+ :param dict readindex:
+ :param dict kmerindex:
+ :param set skipids:
+ """
neighbors = set()
- for origreadid in readids:
- print 'searching ', origreadid
- seq = reads[origreadid]
- for i in range(len(seq) - kmerlength):
- subseq = seq[i:i+kmerlength]
- neighbors.update(index.get(subseq, set()) - skipreads)
- neighbors.update(index.get(revcomp(subseq), set()) - skipreads)
+ for rid in ids_to_search:
+ print 'searching ', rid
+ read = readindex[rid]
+ for i in range(len(read) - kmerlength):
+ kmer = read[i:i + kmerlength]
+ neighbors.update(kmerindex.get(kmer, set()) - skipids)
+ neighbors.update(kmerindex.get(revcomp(kmer), set()) - skipids)
return neighbors
-_revcomp_table = string.maketrans('ACGT', 'TGCA')
-def revcomp(s):
+def revcomp(s, _revcomp_table=string.maketrans('ACGT', 'TGCA')):
+ """return the reverse-complement of s
+ """
return string.translate(s, _revcomp_table)[::-1]
@@ -30,51 +37,40 @@
infile = argv[2]
numhops = int(argv[3])
outname = argv[4]
- except:
+ except IndexError:
print "usage: <prog> origkmer infile numhops outname"
return -1
kmerlength = len(origkmer)
- try:
- print 'reading in', infile + ".reads.pkl"
- reads = pickle.load(open(infile + ".reads.pkl"))
- except:
- print 'reading in', infile
- reads = {line.strip().split('\t')[0] : line.strip().split('\t')[1] for line in open(infile)}
- pickle.dump(reads, open(infile + ".reads.pkl", 'w'), -1)
+ print 'reading in', infile
+ readindex = {rid: read for rid, read in imap(lambda line: line.strip().split('\t'), open(infile))}
- #try:
- # print 'reading', infile + '.index.pkl'
- # index = pickle.load(open(infile + ".index.pkl"))
- #except:
- index = {}
+ kmerindex = {}
print 'building index'
- for readid, line in reads.items():
- for i in range(len(line) - kmerlength):
- index.setdefault(line[i:i+kmerlength], set()).add(readid)
- #pickle.dump(index, open(infile + ".index.pkl", 'w'), -1)
+ for rid, read in readindex.items():
+ for i in range(len(read) - kmerlength):
+ kmerindex.setdefault(read[i:i + kmerlength], set()).add(rid)
outreads = set()
print 'getting matching readids'
curkmer = origkmer
- outreads.update(index.get(curkmer, set()))
- outreads.update(index.get(revcomp(curkmer), set()))
+ outreads.update(kmerindex.get(curkmer, set()))
+ outreads.update(kmerindex.get(revcomp(curkmer), set()))
# get neighbors
new_neighbors = outreads
for i in range(numhops):
print 'hop', i
- new_neighbors = getneighbors(new_neighbors, kmerlength, reads, index, outreads)
+ new_neighbors = getneighbors(new_neighbors, kmerlength, readindex, kmerindex, outreads)
outreads.update(new_neighbors)
print 'found', len(outreads), 'neighbors'
with open(outname, 'w') as outfile:
- for readid in outreads:
- outfile.write(readid + '\t' + reads[readid] + '\n')
-
+ for rid in outreads:
+ outfile.write(rid + '\t' + readindex[rid] + '\n')
if __name__ == '__main__':
diff --git a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java
index ae76da3..7dbfed3 100644
--- a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java
+++ b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java
@@ -141,10 +141,10 @@
private DeploymentId prepareJobs() throws Exception {
URLClassLoader classLoader = (URLClassLoader) this.getClass().getClassLoader();
List<String> jars = new ArrayList<String>();
+ LOG.info("Deploying jar files to NC's");
URL[] urls = classLoader.getURLs();
for (URL url : urls)
if (url.toString().endsWith(".jar")) {
- LOG.info(url.toString());
jars.add(new File(url.getPath()).toString());
}
DeploymentId deploymentId = hcc.deployBinary(jars);
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java
index 373fdde..dcae00a 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java
@@ -1,8 +1,10 @@
package edu.uci.ics.genomix.pregelix.operator;
import java.io.IOException;
+import java.util.Arrays;
import java.util.EnumSet;
import java.util.Iterator;
+import java.util.List;
import java.util.Random;
import java.util.logging.Logger;
@@ -71,6 +73,20 @@
protected EDGETYPE incomingEdgeType; //SplitRepeat and BubbleMerge
protected EDGETYPE outgoingEdgeType; //SplitRepeat and BubbleMerge
+ protected static final List<VKmerBytesWritable> problemKmers = Arrays.asList(
+// new VKmerBytesWritable("CCCGGCCTCCAGCGTGGGATACGCGAAGATGCCGCCGTAGGTGAGAATCTGGTTC"),
+// new VKmerBytesWritable("GCAGGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+// new VKmerBytesWritable("GAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+// new VKmerBytesWritable("GAGCAGGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+// new VKmerBytesWritable("GGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+// new VKmerBytesWritable("AGGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+// new VKmerBytesWritable("GCGACGTGCAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+// new VKmerBytesWritable("GTCAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+ new VKmerBytesWritable("AACACCGAGGGCGTGCCGACGAACCCCGAGGCCTACGCGAAGATGCGCGGCTGGA")
+ );
+ protected static final boolean debug = false;
+ protected boolean verbose = false;
+
protected HashMapWritable<ByteWritable, VLongWritable> counters = new HashMapWritable<ByteWritable, VLongWritable>();
/**
* initiate kmerSize, maxIteration
@@ -81,6 +97,10 @@
if (maxIteration < 0)
maxIteration = Integer.parseInt(getContext().getConfiguration().get(GenomixJobConf.GRAPH_CLEAN_MAX_ITERATIONS));
GenomixJobConf.setGlobalStaticConstants(getContext().getConfiguration());
+
+ verbose = false;
+ for (VKmerBytesWritable problemKmer : problemKmers)
+ verbose |= debug && (getVertexValue().getNode().findEdge(problemKmer) != null || getVertexId().equals(problemKmer));
}
public boolean isHeadNode(){
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java
index bfd35ca..47b71a1 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java
@@ -1,7 +1,11 @@
package edu.uci.ics.genomix.pregelix.operator.pathmerge;
+import java.util.ArrayList;
import java.util.EnumSet;
import java.util.Iterator;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.StringUtils;
import edu.uci.ics.genomix.pregelix.io.VertexValueWritable;
import edu.uci.ics.genomix.pregelix.io.VertexValueWritable.P4State;
@@ -19,6 +23,9 @@
public abstract class BasicPathMergeVertex<V extends VertexValueWritable, M extends PathMergeMessageWritable> extends
BasicGraphCleanVertex<V, M>{
+
+ private static final Logger LOG = Logger.getLogger(BasicPathMergeVertex.class.getName());
+
protected static final boolean isP1 = true;
protected static final boolean isP2 = false;
protected static final boolean isP4 = true;
@@ -448,10 +455,17 @@
for (DIR dir : dirsToRestrict) {
for (EDGETYPE et : dir.edgeType()) {
for (VKmerBytesWritable destId : vertex.getEdgeList(et).getKeys()) {
+ if (debug)
+ for (VKmerBytesWritable problemKmer : problemKmers)
+ verbose |= destId.equals(problemKmer);
outgoingMsg.reset();
outgoingMsg.setFlag(et.mirror().dir().get());
-// LOG.info("send restriction from " + getVertexId() + " to " + destId + " in my " + d + " and their " + DirectionFlag.mirrorEdge(d) + " (" + DirectionFlag.dirFromEdgeType(DirectionFlag.mirrorEdge(d)) + "); I am " + getVertexValue());
+ if (verbose)
+ LOG.fine("send restriction from " + getVertexId() + " to " + destId +
+ " in my " + et + " and their " + et.mirror() +
+ " (" + EDGETYPE.dir(et.mirror()) +
+ "); I am " + getVertexValue());
sendMsg(destId, outgoingMsg);
}
}
@@ -465,10 +479,12 @@
short restrictedDirs = getVertexValue().getState(); // the directions (NEXT/PREVIOUS) that I'm not allowed to merge in
boolean updated = false;
while (msgIterator.hasNext()) {
-// LOG.info("before restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
+ if (verbose)
+ LOG.fine("before restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
incomingMsg = msgIterator.next();
restrictedDirs |= incomingMsg.getFlag();
-// LOG.info("after restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
+ if (verbose)
+ LOG.fine("after restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
updated = true;
}
if (updated) {
@@ -506,10 +522,14 @@
EDGETYPE newEdgetype = EDGETYPE.resolveLinkThroughMiddleNode(updateEdge, mergeEdge);
outgoingMsg.getNode().setEdgeList(newEdgetype, getVertexValue().getEdgeList(mergeEdge)); // copy into outgoingMsg
}
+ if (debug)
+ for (VKmerBytesWritable problemKmer : problemKmers)
+ verbose |= outgoingMsg.getNode().findEdge(problemKmer) != null;
// send the update to all kmers in this list // TODO perhaps we could skip all this if there are no neighbors here
for (VKmerBytesWritable dest : vertex.getEdgeList(updateEdge).getKeys()) {
-// LOG.info("send update message from " + getVertexId() + " to " + dest + ": " + outgoingMsg);
+ if (verbose)
+ LOG.fine("send update message from " + getVertexId() + " to " + dest + ": " + outgoingMsg);
sendMsg(dest, outgoingMsg);
}
}
@@ -519,9 +539,14 @@
VertexValueWritable vertex = getVertexValue();
NodeWritable node = vertex.getNode();
boolean updated = false;
+ ArrayList<PathMergeMessageWritable> allSeenMsgs = new ArrayList<PathMergeMessageWritable>();
while (msgIterator.hasNext()) {
-// LOG.info("before update from neighbor: " + getVertexValue());
incomingMsg = msgIterator.next();
+ if (debug)
+ for (VKmerBytesWritable problemKmer : problemKmers)
+ verbose |= incomingMsg.getNode().findEdge(problemKmer) != null || incomingMsg.getSourceVertexId().equals(problemKmer);
+ if (verbose)
+ LOG.fine("before update from neighbor: " + getVertexValue());
// remove the edge to the node that will merge elsewhere
node.getEdgeList(EDGETYPE.fromByte(incomingMsg.getFlag())).remove(incomingMsg.getSourceVertexId());
// add the node this neighbor will merge into
@@ -529,8 +554,13 @@
node.getEdgeList(edgeType).unionUpdate(incomingMsg.getEdgeList(edgeType));
}
updated = true;
-// LOG.info("after update from neighbor: " + getVertexValue());
+ if (verbose) {
+ LOG.fine("after update from neighbor: " + getVertexValue());
+ allSeenMsgs.add(incomingMsg);
+ }
}
+ if (verbose)
+ LOG.fine("All recieved updates: \n{\n" + StringUtils.join(allSeenMsgs, "\n") + "\n}\n");
if (updated) {
if (DIR.enumSetFromByte(vertex.getState()).containsAll(EnumSet.allOf(DIR.class)))
voteToHalt();
@@ -554,11 +584,13 @@
if (vertex.getDegree(mergeEdgetype.dir()) != 1)
throw new IllegalStateException("Merge attempted in node with degree in " + mergeEdgetype + " direction != 1!\n" + vertex);
VKmerBytesWritable dest = vertex.getEdgeList(mergeEdgetype).get(0).getKey();
-// LOG.info("send merge mesage from " + getVertexId() + " to " + dest + ": " + outgoingMsg + "; my restrictions are: " + DIR.enumSetFromByte(vertex.getState()) + ", their restrictions are: " + DIR.enumSetFromByte(outgoingMsg.getFlag()));
sendMsg(dest, outgoingMsg);
-
-// LOG.info("killing self: " + getVertexId());
deleteVertex(getVertexId());
+
+ if (verbose) {
+ LOG.fine("send merge mesage from " + getVertexId() + " to " + dest + ": " + outgoingMsg + "; my restrictions are: " + DIR.enumSetFromByte(vertex.getState()) + ", their restrictions are: " + DIR.enumSetFromByte(outgoingMsg.getFlag()));
+ LOG.fine("killing self: " + getVertexId());
+ }
}
}
@@ -574,20 +606,27 @@
@SuppressWarnings("unused")
int numMerged = 0;
while (msgIterator.hasNext()) {
-// LOG.info("before merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
incomingMsg = msgIterator.next();
+ if (debug)
+ for (VKmerBytesWritable problemKmer : problemKmers)
+ verbose |= incomingMsg.getNode().findEdge(problemKmer) != null;
+ if (verbose)
+ LOG.fine("before merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
senderEdgetype = EDGETYPE.fromByte(incomingMsg.getFlag());
node.mergeWithNode(senderEdgetype, incomingMsg.getNode());
state |= (byte) (incomingMsg.getFlag() & DIR.MASK); // update incoming restricted directions
numMerged++;
updated = true;
-// LOG.info("after merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
+ if (verbose)
+ LOG.fine("after merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
}
if(isTandemRepeat(getVertexValue())) {
// tandem repeats can't merge anymore; restrict all future merges
state |= DIR.NEXT.get();
state |= DIR.PREVIOUS.get();
updated = true;
+ if (verbose)
+ LOG.fine("recieveMerges is a tandem repeat: " + getVertexId() + " " + getVertexValue());
// updateStatisticsCounter(StatisticsCounter.Num_Cycles);
}
// updateStatisticsCounter(StatisticsCounter.Num_MergedNodes);
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
index dba03fe..824d0da 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
@@ -1,8 +1,13 @@
package edu.uci.ics.genomix.pregelix.operator.pathmerge;
+import java.util.ArrayList;
import java.util.EnumSet;
import java.util.Iterator;
import java.util.Random;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.StringUtils;
+import org.jfree.util.Log;
import edu.uci.ics.genomix.config.GenomixJobConf;
import edu.uci.ics.genomix.pregelix.client.Client;
@@ -13,6 +18,7 @@
import edu.uci.ics.genomix.type.NodeWritable.DIR;
import edu.uci.ics.genomix.type.NodeWritable.EDGETYPE;
import edu.uci.ics.genomix.type.VKmerBytesWritable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
/**
* Graph clean pattern: P4(Smart-algorithm) for path merge
@@ -22,7 +28,7 @@
public class P4ForPathMergeVertex extends
BasicPathMergeVertex<VertexValueWritable, PathMergeMessageWritable> {
-// private static final Logger LOG = Logger.getLogger(P4ForPathMergeVertex.class.getName());
+ private static final Logger LOG = Logger.getLogger(P4ForPathMergeVertex.class.getName());
private static long randSeed = 1; //static for save memory
private float probBeingRandomHead = -1;
@@ -76,14 +82,15 @@
counters.clear();
getVertexValue().getCounters().clear();
}
-
+
protected boolean isNodeRandomHead(VKmerBytesWritable nodeKmer) {
// "deterministically random", based on node id
randGenerator.setSeed((randSeed ^ nodeKmer.hashCode()) * 10000 * getSuperstep());//randSeed + nodeID.hashCode()
for(int i = 0; i < 500; i++)
randGenerator.nextFloat();
boolean isHead = randGenerator.nextFloat() < probBeingRandomHead;
-// LOG.info("randomHead: " + nodeKmer + "=" + isHead);
+ if (verbose)
+ LOG.fine("randomHead: " + nodeKmer + "=" + isHead);
return isHead;
}
@@ -157,15 +164,17 @@
}
}
}
-// if ((getVertexValue().getState() & P4State.MERGE) == 0) {
-// LOG.info("No merge for " + getVertexId());
-// } else {
-// LOG.info("Merge from " + getVertexId() + " towards " + (getVertexValue().getState() & DirectionFlag.DIR_MASK) + "; node is " + getVertexValue());
-// }
+ if (verbose) {
+// if ((getVertexValue().getState() & P4State.MERGE) == 0) {
+// LOG.fine("No merge for " + getVertexId());
+// } else {
+ LOG.fine("Merge from " + getVertexId() + " towards " + (EDGETYPE.fromByte(getVertexValue().getState())) + "; node is " + getVertexValue());
+// }
+ }
}
@Override
- public void compute(Iterator<PathMergeMessageWritable> msgIterator) {
+ public void compute(Iterator<PathMergeMessageWritable> msgIterator) throws HyracksDataException {
initVertex();
if (getSuperstep() == 1) {
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 09eeab6..c72ced1 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
@@ -278,6 +278,9 @@
removePendingThread(ct);
}
if (!exceptions.isEmpty()) {
+ for (Exception e : exceptions) {
+ e.printStackTrace();
+ }
NodeControllerService ncs = joblet.getNodeController();
ExceptionUtils.setNodeIds(exceptions, ncs.getId());
ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, exceptions));
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
index 04b91b2..31cd29b 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
@@ -43,6 +43,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
@@ -53,8 +54,8 @@
private final MarshalledWritable<Configuration> config;
private final IInputSplitProviderFactory factory;
- public MapperOperatorDescriptor(IOperatorDescriptorRegistry spec, int jobId, MarshalledWritable<Configuration> config,
- IInputSplitProviderFactory factory) throws HyracksDataException {
+ public MapperOperatorDescriptor(IOperatorDescriptorRegistry spec, int jobId,
+ MarshalledWritable<Configuration> config, IInputSplitProviderFactory factory) throws HyracksDataException {
super(spec, 0, 1);
this.jobId = jobId;
this.config = config;
@@ -94,7 +95,7 @@
public void initBlock(int blockId) throws HyracksDataException {
runGen = new ExternalSortRunGenerator(ctx, new int[] { 0 }, null, comparatorFactories,
- helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit);
+ helper.getMapOutputRecordDescriptorWithoutExtraFields(), Algorithm.MERGE_SORT, framesLimit);
this.blockId = blockId;
}
@@ -114,7 +115,8 @@
runGen.nextFrame(frame);
fta.reset(frame, true);
if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + frame.capacity() + ")");
+ throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
+ + frame.capacity() + ")");
}
}
}
@@ -224,7 +226,7 @@
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getFrameSorter(),
- runGen.getRuns(), new int[] { 0 }, comparators,
+ runGen.getRuns(), new int[] { 0 }, comparators, null,
helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit, delegatingWriter);
merger.process();
}
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 30ba3ec..3bb78f9 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
@@ -108,7 +108,7 @@
runs.add(rfw.createReader());
}
RunFileWriter rfw = new RunFileWriter(outFile, ctx.getIOManager());
- ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators,
+ ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators, null,
recordDescriptor, framesLimit, rfw);
merger.process();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
index 235b0d0..2dda9cc 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
@@ -21,6 +21,7 @@
import edu.uci.ics.hyracks.api.comm.IFrameReader;
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.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
@@ -31,18 +32,21 @@
private final int nSenders;
private final int[] sortFields;
private final IBinaryComparator[] comparators;
+ private final INormalizedKeyComputer nmkComputer;
private final RecordDescriptor recordDescriptor;
private final IPartitionBatchManager pbm;
private RunMergingFrameReader merger;
public SortMergeFrameReader(IHyracksTaskContext ctx, int maxConcurrentMerges, int nSenders, int[] sortFields,
- IBinaryComparator[] comparators, RecordDescriptor recordDescriptor, IPartitionBatchManager pbm) {
+ IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDescriptor,
+ IPartitionBatchManager pbm) {
this.ctx = ctx;
this.maxConcurrentMerges = maxConcurrentMerges;
this.nSenders = nSenders;
this.sortFields = sortFields;
this.comparators = comparators;
+ this.nmkComputer = nmkComputer;
this.recordDescriptor = recordDescriptor;
this.pbm = pbm;
}
@@ -57,7 +61,7 @@
List<IFrameReader> batch = new ArrayList<IFrameReader>();
pbm.getNextBatch(batch, nSenders);
merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
- comparators, recordDescriptor);
+ comparators, nmkComputer, recordDescriptor);
} else {
// multi level merge.
throw new HyracksDataException("Not yet supported");
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
index 32269bd..1cf402b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -23,6 +23,8 @@
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.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -39,19 +41,23 @@
private final ITuplePartitionComputerFactory tpcf;
private final int[] sortFields;
private final IBinaryComparatorFactory[] comparatorFactories;
+ private final INormalizedKeyComputerFactory nkcFactory;
private final boolean stable;
- public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
- int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
- this(spec, tpcf, sortFields, comparatorFactories, false);
+ public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+ ITuplePartitionComputerFactory tpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory nkcFactory) {
+ this(spec, tpcf, sortFields, comparatorFactories, nkcFactory, false);
}
- public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
- int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, boolean stable) {
+ public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+ ITuplePartitionComputerFactory tpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory nkcFactory, boolean stable) {
super(spec);
this.tpcf = tpcf;
this.sortFields = sortFields;
this.comparatorFactories = comparatorFactories;
+ this.nkcFactory = nkcFactory;
this.stable = stable;
}
@@ -71,9 +77,10 @@
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
+ INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
- sortFields, comparators, recordDesc, pbm);
+ sortFields, comparators, nmkComputer, recordDesc, pbm);
BitSet expectedPartitions = new BitSet();
expectedPartitions.set(0, nProducerPartitions);
return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
index 21b3b60..bb3d0f3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
@@ -25,6 +25,8 @@
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.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -47,6 +49,7 @@
private final Object stateId;
private final int[] keyFields;
private final IBinaryComparator[] comparators;
+ private final INormalizedKeyComputer nmkComputer;
private final AggregateState aggregateState;
private final ArrayTupleBuilder tupleBuilder;
private final int[] storedKeys;
@@ -76,7 +79,7 @@
private final FrameTupleAccessor outFrameAccessor;
ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
- IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+ IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nmkFactory, int[] keyFields,
IAggregatorDescriptorFactory mergerFactory, boolean isOutputSorted, int framesLimit,
RecordDescriptor outRecordDescriptor) throws HyracksDataException {
this.stateId = stateId;
@@ -85,6 +88,7 @@
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
+ this.nmkComputer = nmkFactory == null ? null : nmkFactory.createNormalizedKeyComputer();
int[] keyFieldsInPartialResults = new int[keyFields.length];
for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
keyFieldsInPartialResults[i] = i;
@@ -181,7 +185,7 @@
FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), outRecordDescriptor,
- runNumber, comparator);
+ runNumber, comparator, keyFields, nmkComputer);
/**
* current tuple index in each run
*/
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
index b914702..9296ffa 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
@@ -130,8 +130,8 @@
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
return new ExternalGroupMergeOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
- AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, keyFields, mergerFactory, isOutputSorted,
- framesLimit, recordDescriptors[0]);
+ AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, firstNormalizerFactory, keyFields,
+ mergerFactory, isOutputSorted, framesLimit, recordDescriptors[0]);
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java
new file mode 100644
index 0000000..5c96bd0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java
@@ -0,0 +1,21 @@
+/*
+ * 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.dataflow.std.sort;
+
+public enum Algorithm {
+ QUICK_SORT,
+ MERGE_SORT
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index f80a82c..e1315e7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -28,6 +28,7 @@
import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -51,6 +52,15 @@
private final IBinaryComparatorFactory[] comparatorFactories;
private final int framesLimit;
+ private Algorithm alg = Algorithm.MERGE_SORT;
+
+ public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, Algorithm alg) {
+ this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+ this.alg = alg;
+ }
+
public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
this(spec, framesLimit, sortFields, null, comparatorFactories, recordDescriptor);
@@ -86,7 +96,7 @@
public static class SortTaskState extends AbstractStateObject {
private List<IFrameReader> runs;
- private FrameSorter frameSorter;
+ private IFrameSorter frameSorter;
public SortTaskState() {
}
@@ -122,7 +132,7 @@
@Override
public void open() throws HyracksDataException {
runGen = new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
- comparatorFactories, recordDescriptors[0], framesLimit);
+ comparatorFactories, recordDescriptors[0], alg, framesLimit);
runGen.open();
}
@@ -166,14 +176,16 @@
SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
SORT_ACTIVITY_ID), partition));
List<IFrameReader> runs = state.runs;
- FrameSorter frameSorter = state.frameSorter;
+ IFrameSorter frameSorter = state.frameSorter;
IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
+ INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+ : firstKeyNormalizerFactory.createNormalizedKeyComputer();
int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
- comparators, recordDescriptors[0], necessaryFrames, writer);
+ comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
merger.process();
}
};
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 b149e30..3736fca 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
@@ -30,15 +30,21 @@
public class ExternalSortRunGenerator implements IFrameWriter {
private final IHyracksTaskContext ctx;
- private final FrameSorter frameSorter;
+ private final IFrameSorter frameSorter;
private final List<IFrameReader> runs;
private final int maxSortFrames;
public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDesc, int framesLimit) throws HyracksDataException {
+ RecordDescriptor recordDesc, Algorithm alg, int framesLimit) throws HyracksDataException {
this.ctx = ctx;
- frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
+ if (alg == Algorithm.MERGE_SORT) {
+ frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+ recordDesc);
+ } else {
+ frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+ recordDesc);
+ }
runs = new LinkedList<IFrameReader>();
maxSortFrames = framesLimit - 1;
}
@@ -87,7 +93,7 @@
public void fail() throws HyracksDataException {
}
- public FrameSorter getFrameSorter() {
+ public IFrameSorter getFrameSorter() {
return frameSorter;
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 510dfd6..eaf4162 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -23,6 +23,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.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -59,6 +60,7 @@
private final List<IFrameReader> runs;
private final int[] sortFields;
private final IBinaryComparator[] comparators;
+ private final INormalizedKeyComputer nmkComputer;
private final RecordDescriptor recordDesc;
private final int framesLimit;
private final IFrameWriter writer;
@@ -66,8 +68,8 @@
private ByteBuffer outFrame;
private FrameTupleAppender outFrameAppender;
- private FrameSorter frameSorter; // Used in External sort, no replacement
- // selection
+ private IFrameSorter frameSorter; // Used in External sort, no replacement
+ // selection
private FrameTupleAccessor outFrameAccessor; // Used in External sort, with
// replacement selection
private final int outputLimit; // Used in External sort, with replacement
@@ -76,14 +78,15 @@
// selection and limit on output size
// Constructor for external sort, no replacement selection
- public ExternalSortRunMerger(IHyracksTaskContext ctx, FrameSorter frameSorter, List<IFrameReader> runs,
- int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit,
- IFrameWriter writer) {
+ public ExternalSortRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+ int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+ RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
this.ctx = ctx;
this.frameSorter = frameSorter;
this.runs = new LinkedList<IFrameReader>(runs);
this.sortFields = sortFields;
this.comparators = comparators;
+ this.nmkComputer = nmkComputer;
this.recordDesc = recordDesc;
this.framesLimit = framesLimit;
this.writer = writer;
@@ -92,11 +95,13 @@
// Constructor for external sort with replacement selection
public ExternalSortRunMerger(IHyracksTaskContext ctx, int outputLimit, List<IFrameReader> runs, int[] sortFields,
- IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
+ IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
+ int framesLimit, IFrameWriter writer) {
this.ctx = ctx;
this.runs = new LinkedList<IFrameReader>(runs);
this.sortFields = sortFields;
this.comparators = comparators;
+ this.nmkComputer = nmkComputer;
this.recordDesc = recordDesc;
this.framesLimit = framesLimit;
this.writer = writer;
@@ -162,7 +167,7 @@
private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, comparators,
- recordDesc);
+ nmkComputer, recordDesc);
merger.open();
try {
while (merger.nextFrame(outFrame)) {
@@ -263,7 +268,7 @@
runCursors[i] = runs.get(i);
}
RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
- comparators, recordDesc);
+ comparators, nmkComputer, recordDesc);
merger.open();
try {
while (merger.nextFrame(outFrame)) {
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/FrameSorterMergeSort.java
similarity index 97%
rename from hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
rename to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index a6bb4e2..cc0f1ef 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/FrameSorterMergeSort.java
@@ -31,7 +31,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
-public class FrameSorter {
+public class FrameSorterMergeSort implements IFrameSorter {
private final IHyracksTaskContext ctx;
private final int[] sortFields;
private final INormalizedKeyComputer nkc;
@@ -50,7 +50,7 @@
private int[] tPointersTemp;
private int tupleCount;
- public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+ public FrameSorterMergeSort(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor) throws HyracksDataException {
this.ctx = ctx;
@@ -69,15 +69,18 @@
dataFrameCount = 0;
}
+ @Override
public void reset() {
dataFrameCount = 0;
tupleCount = 0;
}
+ @Override
public int getFrameCount() {
return dataFrameCount;
}
+ @Override
public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
ByteBuffer copyFrame;
if (dataFrameCount == buffers.size()) {
@@ -90,6 +93,7 @@
++dataFrameCount;
}
+ @Override
public void sortFrames() {
int nBuffers = dataFrameCount;
tupleCount = 0;
@@ -123,6 +127,7 @@
}
}
+ @Override
public void flushFrames(IFrameWriter writer) throws HyracksDataException {
appender.reset(outFrame, true);
for (int ptr = 0; ptr < tupleCount; ++ptr) {
@@ -241,6 +246,7 @@
return 0;
}
+ @Override
public void close() {
this.buffers.clear();
}
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/FrameSorterQuickSort.java
similarity index 71%
copy from hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
copy to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
index a6bb4e2..083f4a7 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/FrameSorterQuickSort.java
@@ -29,9 +29,8 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
-public class FrameSorter {
+public class FrameSorterQuickSort implements IFrameSorter {
private final IHyracksTaskContext ctx;
private final int[] sortFields;
private final INormalizedKeyComputer nkc;
@@ -47,10 +46,9 @@
private int dataFrameCount;
private int[] tPointers;
- private int[] tPointersTemp;
private int tupleCount;
- public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+ public FrameSorterQuickSort(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor) throws HyracksDataException {
this.ctx = ctx;
@@ -69,15 +67,18 @@
dataFrameCount = 0;
}
+ @Override
public void reset() {
dataFrameCount = 0;
tupleCount = 0;
}
+ @Override
public int getFrameCount() {
return dataFrameCount;
}
+ @Override
public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
ByteBuffer copyFrame;
if (dataFrameCount == buffers.size()) {
@@ -90,6 +91,7 @@
++dataFrameCount;
}
+ @Override
public void sortFrames() {
int nBuffers = dataFrameCount;
tupleCount = 0;
@@ -118,11 +120,11 @@
}
}
if (tupleCount > 0) {
- tPointersTemp = new int[tPointers.length];
- sort(0, tupleCount);
+ sort(tPointers, 0, tupleCount);
}
}
+ @Override
public void flushFrames(IFrameWriter writer) throws HyracksDataException {
appender.reset(outFrame, true);
for (int ptr = 0; ptr < tupleCount; ++ptr) {
@@ -145,73 +147,75 @@
}
}
- private void sort(int offset, int length) {
- int step = 1;
- int len = length;
- int end = offset + len;
- /** bottom-up merge */
- while (step < len) {
- /** merge */
- for (int i = offset; i < end; i += 2 * step) {
- int next = i + step;
- if (next < end) {
- merge(i, next, step, Math.min(step, end - next));
- } else {
- System.arraycopy(tPointers, i * 4, tPointersTemp, i * 4, (end - i) * 4);
+ private void sort(int[] tPointers, int offset, int length) {
+ int m = offset + (length >> 1);
+ int mi = tPointers[m * 4];
+ int mj = tPointers[m * 4 + 1];
+ int mv = tPointers[m * 4 + 3];
+
+ int a = offset;
+ int b = a;
+ int c = offset + length - 1;
+ int d = c;
+ while (true) {
+ while (b <= c) {
+ int cmp = compare(tPointers, b, mi, mj, mv);
+ if (cmp > 0) {
+ break;
}
+ if (cmp == 0) {
+ swap(tPointers, a++, b);
+ }
+ ++b;
}
- /** prepare next phase merge */
- step *= 2;
- int[] tmp = tPointersTemp;
- tPointersTemp = tPointers;
- tPointers = tmp;
- }
- }
-
- /** Merge two subarrays into one */
- private void merge(int start1, int start2, int len1, int len2) {
- int targetPos = start1;
- int pos1 = start1;
- int pos2 = start2;
- int end1 = start1 + len1 - 1;
- int end2 = start2 + len2 - 1;
- while (pos1 <= end1 && pos2 <= end2) {
- int cmp = compare(pos1, pos2);
- if (cmp <= 0) {
- copy(pos1, targetPos);
- pos1++;
- } else {
- copy(pos2, targetPos);
- pos2++;
+ while (c >= b) {
+ int cmp = compare(tPointers, c, mi, mj, mv);
+ if (cmp < 0) {
+ break;
+ }
+ if (cmp == 0) {
+ swap(tPointers, c, d--);
+ }
+ --c;
}
- targetPos++;
+ if (b > c)
+ break;
+ swap(tPointers, b++, c--);
}
- if (pos1 <= end1) {
- int rest = end1 - pos1 + 1;
- System.arraycopy(tPointers, pos1 * 4, tPointersTemp, targetPos * 4, rest * 4);
+
+ int s;
+ int n = offset + length;
+ s = Math.min(a - offset, b - a);
+ vecswap(tPointers, offset, b - s, s);
+ s = Math.min(d - c, n - d - 1);
+ vecswap(tPointers, b, n - s, s);
+
+ if ((s = b - a) > 1) {
+ sort(tPointers, offset, s);
}
- if (pos2 <= end2) {
- int rest = end2 - pos2 + 1;
- System.arraycopy(tPointers, pos2 * 4, tPointersTemp, targetPos * 4, rest * 4);
+ if ((s = d - c) > 1) {
+ sort(tPointers, n - s, s);
}
}
- private void copy(int src, int dest) {
- tPointersTemp[dest * 4] = tPointers[src * 4];
- tPointersTemp[dest * 4 + 1] = tPointers[src * 4 + 1];
- tPointersTemp[dest * 4 + 2] = tPointers[src * 4 + 2];
- tPointersTemp[dest * 4 + 3] = tPointers[src * 4 + 3];
+ private void swap(int x[], int a, int b) {
+ for (int i = 0; i < 4; ++i) {
+ int t = x[a * 4 + i];
+ x[a * 4 + i] = x[b * 4 + i];
+ x[b * 4 + i] = t;
+ }
}
- private int compare(int tp1, int tp2) {
+ private void vecswap(int x[], int a, int b, int n) {
+ for (int i = 0; i < n; i++, a++, b++) {
+ swap(x, a, b);
+ }
+ }
+
+ private int compare(int[] tPointers, int tp1, int tp2i, int tp2j, int tp2v) {
int i1 = tPointers[tp1 * 4];
int j1 = tPointers[tp1 * 4 + 1];
int v1 = tPointers[tp1 * 4 + 3];
-
- int tp2i = tPointers[tp2 * 4];
- int tp2j = tPointers[tp2 * 4 + 1];
- int tp2v = tPointers[tp2 * 4 + 3];
-
if (v1 != tp2v) {
return ((((long) v1) & 0xffffffffL) < (((long) tp2v) & 0xffffffffL)) ? -1 : 1;
}
@@ -225,12 +229,12 @@
fta2.reset(buf2);
for (int f = 0; f < comparators.length; ++f) {
int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf1.array(), j1 + (fIdx - 1) * 4);
- int f1End = IntSerDeUtils.getInt(buf1.array(), j1 + fIdx * 4);
+ int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
+ int f1End = buf1.getInt(j1 + fIdx * 4);
int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf2.array(), j2 + (fIdx - 1) * 4);
- int f2End = IntSerDeUtils.getInt(buf2.array(), j2 + fIdx * 4);
+ int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
+ int f2End = buf2.getInt(j2 + fIdx * 4);
int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
int l2 = f2End - f2Start;
int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
@@ -241,7 +245,8 @@
return 0;
}
+ @Override
public void close() {
this.buffers.clear();
}
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
new file mode 100644
index 0000000..6778852
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameSorter {
+
+ public void reset();
+
+ public int getFrameCount();
+
+ public void insertFrame(ByteBuffer buffer) throws HyracksDataException;
+
+ public void sortFrames();
+
+ public void flushFrames(IFrameWriter writer) throws HyracksDataException;
+
+ public void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 04c82af..6fa21b5 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -77,7 +77,7 @@
}
public static class SortTaskState extends AbstractStateObject {
- private FrameSorter frameSorter;
+ private FrameSorterMergeSort frameSorter;
public SortTaskState() {
}
@@ -111,7 +111,7 @@
@Override
public void open() throws HyracksDataException {
state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
- state.frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
+ state.frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory,
comparatorFactories, recordDescriptors[0]);
state.frameSorter.reset();
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
index bff2e21..ef1ae88 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
@@ -28,6 +28,7 @@
import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -201,9 +202,11 @@
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
+ INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+ : firstKeyNormalizerFactory.createNormalizedKeyComputer();
int necessaryFrames = Math.min(runs.size() + 2, memSize);
ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, outputLimit, runs, sortFields,
- comparators, recordDescriptors[0], necessaryFrames, writer);
+ comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
merger.processWithReplacementSelection();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
index 16b3c12..00fbe9b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -22,6 +22,7 @@
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
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.INormalizedKeyComputer;
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.FrameTupleAccessor;
@@ -35,6 +36,7 @@
private final List<ByteBuffer> inFrames;
private final int[] sortFields;
private final IBinaryComparator[] comparators;
+ private final INormalizedKeyComputer nmkComputer;
private final RecordDescriptor recordDesc;
private final FrameTupleAppender outFrameAppender;
private ReferencedPriorityQueue topTuples;
@@ -42,12 +44,14 @@
private FrameTupleAccessor[] tupleAccessors;
public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
- int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc) {
+ int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+ RecordDescriptor recordDesc) {
this.ctx = ctx;
this.runCursors = runCursors;
this.inFrames = inFrames;
this.sortFields = sortFields;
this.comparators = comparators;
+ this.nmkComputer = nmkComputer;
this.recordDesc = recordDesc;
outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
}
@@ -56,7 +60,8 @@
public void open() throws HyracksDataException {
tupleAccessors = new FrameTupleAccessor[runCursors.length];
Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
- topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator);
+ topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator,
+ sortFields, nmkComputer);
tupleIndexes = new int[runCursors.length];
for (int i = 0; i < runCursors.length; i++) {
tupleIndexes[i] = 0;
@@ -143,21 +148,25 @@
private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
return new Comparator<ReferenceEntry>() {
public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+ int nmk1 = tp1.getNormalizedKey();
+ int nmk2 = tp1.getNormalizedKey();
+ if (nmk1 > nmk2) {
+ return 1;
+ }
+ if (nmk1 < nmk2) {
+ return -1;
+ }
+
FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
- int j1 = tp1.getTupleIndex();
- int j2 = tp2.getTupleIndex();
byte[] b1 = fta1.getBuffer().array();
byte[] b2 = fta2.getBuffer().array();
+ int[] tPointers1 = tp1.getTPointers();
+ int[] tPointers2 = tp2.getTPointers();
+
for (int f = 0; f < sortFields.length; ++f) {
- int fIdx = sortFields[f];
- int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
- + fta1.getFieldStartOffset(j1, fIdx);
- int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
- int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
- + fta2.getFieldStartOffset(j2, fIdx);
- int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
- int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+ int c = comparators[f].compare(b1, tPointers1[2 * f + 1], tPointers1[2 * f + 2], b2,
+ tPointers2[2 * f + 1], tPointers2[2 * f + 2]);
if (c != 0) {
return c;
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
index c1efa91..c06b50c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -14,18 +14,24 @@
*/
package edu.uci.ics.hyracks.dataflow.std.util;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
public class ReferenceEntry {
private final int runid;
private FrameTupleAccessor acccessor;
private int tupleIndex;
+ private int[] tPointers;
- public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex) {
+ public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+ INormalizedKeyComputer nmkComputer) {
super();
this.runid = runid;
this.acccessor = fta;
- this.tupleIndex = tupleIndex;
+ this.tPointers = new int[1 + 2 * keyFields.length];
+ if (fta != null) {
+ initTPointer(fta, tupleIndex, keyFields, nmkComputer);
+ }
}
public int getRunid() {
@@ -40,11 +46,38 @@
this.acccessor = fta;
}
+ public int[] getTPointers() {
+ return tPointers;
+ }
+
public int getTupleIndex() {
return tupleIndex;
}
- public void setTupleIndex(int tupleIndex) {
+ public int getNormalizedKey() {
+ return tPointers[0];
+ }
+
+ public void setTupleIndex(int tupleIndex, int[] keyFields, INormalizedKeyComputer nmkComputer) {
+ initTPointer(acccessor, tupleIndex, keyFields, nmkComputer);
+ }
+
+ private void initTPointer(FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+ INormalizedKeyComputer nmkComputer) {
this.tupleIndex = tupleIndex;
+ byte[] b1 = fta.getBuffer().array();
+ for (int f = 0; f < keyFields.length; ++f) {
+ int fIdx = keyFields[f];
+ tPointers[2 * f + 1] = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength()
+ + fta.getFieldStartOffset(tupleIndex, fIdx);
+ tPointers[2 * f + 2] = fta.getFieldEndOffset(tupleIndex, fIdx) - fta.getFieldStartOffset(tupleIndex, fIdx);
+ if (f == 0) {
+ if (nmkComputer != null) {
+ tPointers[0] = nmkComputer.normalize(b1, tPointers[1], tPointers[2]);
+ } else {
+ tPointers[0] = 0;
+ }
+ }
+ }
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
index 7767ace..225f583 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -18,6 +18,7 @@
import java.util.BitSet;
import java.util.Comparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -30,21 +31,25 @@
private int nItems;
private final Comparator<ReferenceEntry> comparator;
+ private final INormalizedKeyComputer nmkComputer;
+ private final int[] keyFields;
public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
- Comparator<ReferenceEntry> comparator) {
+ Comparator<ReferenceEntry> comparator, int[] keyFields, INormalizedKeyComputer nmkComputer) {
this.frameSize = frameSize;
this.recordDescriptor = recordDescriptor;
if (initSize < 1)
throw new IllegalArgumentException();
this.comparator = comparator;
+ this.nmkComputer = nmkComputer;
+ this.keyFields = keyFields;
nItems = initSize;
size = (initSize + 1) & 0xfffffffe;
entries = new ReferenceEntry[size];
runAvail = new BitSet(size);
runAvail.set(0, initSize, true);
for (int i = 0; i < size; i++) {
- entries[i] = new ReferenceEntry(i, null, -1);
+ entries[i] = new ReferenceEntry(i, null, -1, keyFields, nmkComputer);
}
}
@@ -71,7 +76,7 @@
entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
}
entry.getAccessor().reset(fta.getBuffer());
- entry.setTupleIndex(tIndex);
+ entry.setTupleIndex(tIndex, keyFields, nmkComputer);
add(entry);
return entry.getRunid();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 380ba29..a2ef99a 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -31,6 +31,7 @@
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -93,8 +94,8 @@
PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
new IBinaryComparatorFactory[] {
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
- printer, 0);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
runTest(spec);
}
@@ -148,8 +149,8 @@
PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
new IBinaryComparatorFactory[] {
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0, filter,
- 0);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, filter, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), filter, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 48db176..faa55e8 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -31,6 +31,7 @@
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -89,7 +90,8 @@
new int[] { 1 }, new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }), new int[] { 1 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) }), sorter, 0, printer, 0);
+ .of(UTF8StringPointable.FACTORY) }, new UTF8StringNormalizedKeyComputerFactory()),
+ sorter, 0, printer, 0);
runTest(spec);
}
@@ -138,8 +140,8 @@
PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
new IBinaryComparatorFactory[] {
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
- printer, 0);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
runTest(spec);
}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
index c61c15b..61c5a1c 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -156,8 +156,8 @@
jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
- new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
- 0, writeOperator, 0);
+ new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, null),
+ sortOperator, 0, writeOperator, 0);
jobSpec.addRoot(writeOperator);
IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
index 3e43fb1..baa4dc7 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -163,8 +163,8 @@
jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
- new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
- 0, writeOperator, 0);
+ new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, null),
+ sortOperator, 0, writeOperator, 0);
jobSpec.addRoot(writeOperator);
IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
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 c14f23a..3e4ecf8 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
@@ -264,7 +264,7 @@
throw new TreeIndexException("Cannot bulk-load a non-empty tree.");
}
- this.cmp = MultiComparator.createIgnoreFieldLength(cmpFactories);
+ this.cmp = MultiComparator.create(cmpFactories);
leafFrame.setMultiComparator(cmp);
interiorFrame.setMultiComparator(cmp);
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 35c9bcb..8a3c313 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
@@ -23,7 +23,7 @@
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.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -35,17 +35,17 @@
public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ opTrackerFactory, ioScheduler, ioOpCallbackFactory);
}
public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
+ ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider);
+ ioScheduler, ioOpCallbackFactory);
}
@Override
@@ -55,6 +55,6 @@
opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
- ioOpCallbackProvider);
+ ioOpCallbackFactory.createIOOperationCallback());
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
index f4be88f..104a70d 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -31,10 +31,10 @@
public LSMBTreeDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
double bloomFilterFalsePositiveRate) {
super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerFactory, ioSchedulerProvider,
- ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+ ioOpCallbackFactory, bloomFilterFalsePositiveRate);
}
@Override
@@ -43,6 +43,6 @@
return new LSMBTreeDataflowHelper(opDesc, ctx, partition,
virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
- ioOpCallbackProvider);
+ ioOpCallbackFactory);
}
}
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 c8ed0d0..33ae7bb 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
@@ -92,9 +92,9 @@
TreeIndexFactory<BTree> bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory,
double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount,
IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) {
super(virtualBufferCaches, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback);
int i = 0;
for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
LSMBTreeMemoryComponent mutableComponent = new LSMBTreeMemoryComponent(new BTree(virtualBufferCache,
@@ -165,7 +165,7 @@
if (flushOnExit) {
BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
- ioOpCallbackProvider.getIOOperationCallback(this));
+ ioOpCallback);
ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
accessor.scheduleFlush(cb);
try {
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 f43c8a6..8b363ff 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
@@ -33,7 +33,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -49,7 +49,7 @@
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallback ioOpCallback) {
LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
cmpFactories.length, false);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -78,7 +78,7 @@
LSMBTree lsmTree = new LSMBTree(virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length, cmpFactories, mergePolicy,
- opTracker, ioScheduler, ioOpCallbackProvider);
+ opTracker, ioScheduler, ioOpCallback);
return lsmTree;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index 7086b59..9af08fa 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -17,12 +17,36 @@
import java.util.List;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
public interface ILSMIOOperationCallback {
- public void beforeOperation() throws HyracksDataException;
- public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ /**
+ * This method is called on an IO operation sometime before the operation is executed.
+ * (i.e. IO operations could be flush or merge operations.)
+ */
+ public void beforeOperation(LSMOperationType opType) throws HyracksDataException;
+
+ /**
+ * This method is called on an IO operation sometime after the operation was completed.
+ * (i.e. IO operations could be flush or merge operations.)
+ *
+ * @param oldComponents
+ * @param newComponent
+ * @throws HyracksDataException
+ */
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException;
- public void afterFinalize(ILSMComponent newComponent) throws HyracksDataException;
+ /**
+ * This method is called on an IO operation when the operation needs any cleanup works
+ * regardless that the IO operation was executed or not. Once the IO operation is executed,
+ * this method should be called after ILSMIOOperationCallback.afterOperation() was called.
+ *
+ * @param newComponent
+ * @throws HyracksDataException
+ */
+ public void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) throws HyracksDataException;
+
+ public void setNumOfMutableComponents(int count);
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index 6dbce32..804142e 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -17,5 +17,5 @@
import java.io.Serializable;
public interface ILSMIOOperationCallbackFactory extends Serializable {
- public ILSMIOOperationCallback createIOOperationCallback(Object syncObj);
+ public ILSMIOOperationCallback createIOOperationCallback();
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index 416a15b..d37a105 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -41,6 +41,8 @@
public ILSMOperationTracker getOperationTracker();
public ILSMIOOperationScheduler getIOScheduler();
+
+ public ILSMIOOperationCallback getIOOperationCallback();
public List<ILSMComponent> getImmutableComponents();
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
index b1bf6b4..e277edc 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -20,7 +20,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -36,26 +36,26 @@
protected final ILSMMergePolicy mergePolicy;
protected final ILSMIOOperationScheduler ioScheduler;
protected final ILSMOperationTrackerProvider opTrackerFactory;
- protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ opTrackerFactory, ioScheduler, ioOpCallbackFactory);
}
public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
super(opDesc, ctx, partition);
this.virtualBufferCaches = virtualBufferCaches;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.mergePolicy = mergePolicy;
this.opTrackerFactory = opTrackerFactory;
this.ioScheduler = ioScheduler;
- this.ioOpCallbackProvider = ioOpCallbackProvider;
+ this.ioOpCallbackFactory = ioOpCallbackFactory;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
index 205bf27..2c082bb 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -29,18 +29,18 @@
protected final ILSMMergePolicyProvider mergePolicyProvider;
protected final ILSMOperationTrackerProvider opTrackerFactory;
protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
- protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
protected final double bloomFilterFalsePositiveRate;
public AbstractLSMIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
double bloomFilterFalsePositiveRate) {
this.virtualBufferCacheProvider = virtualBufferCacheProvider;
this.mergePolicyProvider = mergePolicyProvider;
this.opTrackerFactory = opTrackerFactory;
this.ioSchedulerProvider = ioSchedulerProvider;
- this.ioOpCallbackProvider = ioOpCallbackProvider;
+ this.ioOpCallbackFactory = ioOpCallbackFactory;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 794f440..44bcfc2 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -26,6 +26,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
@@ -42,7 +43,7 @@
protected final ILSMHarness lsmHarness;
protected final ILSMIOOperationScheduler ioScheduler;
- protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ protected final ILSMIOOperationCallback ioOpCallback;
// In-memory components.
protected final List<ILSMComponent> memoryComponents;
@@ -63,14 +64,15 @@
public AbstractLSMIndex(List<IVirtualBufferCache> virtualBufferCaches, IBufferCache diskBufferCache,
ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) {
this.virtualBufferCaches = virtualBufferCaches;
this.diskBufferCache = diskBufferCache;
this.diskFileMapProvider = diskFileMapProvider;
this.fileManager = fileManager;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.ioScheduler = ioScheduler;
- this.ioOpCallbackProvider = ioOpCallbackProvider;
+ this.ioOpCallback = ioOpCallback;
+ this.ioOpCallback.setNumOfMutableComponents(virtualBufferCaches.size());
lsmHarness = new LSMHarness(this, mergePolicy, opTracker);
isActivated = false;
diskComponents = new LinkedList<ILSMComponent>();
@@ -179,6 +181,11 @@
}
@Override
+ public ILSMIOOperationCallback getIOOperationCallback() {
+ return ioOpCallback;
+ }
+
+ @Override
public IBufferCache getBufferCache() {
return diskBufferCache;
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
index a7ca95f..fefe812 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
@@ -38,20 +38,25 @@
}
@Override
- public void beforeOperation() throws HyracksDataException {
- wrappedCallback.beforeOperation();
+ public void beforeOperation(LSMOperationType opType) throws HyracksDataException {
+ wrappedCallback.beforeOperation(opType);
}
@Override
- public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
- wrappedCallback.afterOperation(oldComponents, newComponent);
+ wrappedCallback.afterOperation(opType, oldComponents, newComponent);
}
@Override
- public synchronized void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
- wrappedCallback.afterFinalize(newComponent);
+ public synchronized void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) throws HyracksDataException {
+ wrappedCallback.afterFinalize(opType, newComponent);
notifyAll();
notified = true;
}
+
+ @Override
+ public void setNumOfMutableComponents(int count) {
+ wrappedCallback.setNumOfMutableComponents(count);
+ }
}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 145bfe9..ca775b7 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -118,6 +118,7 @@
// Check if there is any action that is needed to be taken based on the operation type
switch (opType) {
case FLUSH:
+ lsmIndex.getIOOperationCallback().beforeOperation(LSMOperationType.FLUSH);
// Changing the flush status should *always* precede changing the mutable component.
lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
lsmIndex.changeMutableComponent();
@@ -125,6 +126,8 @@
// again if they can grab and enter the mutable component.
opTracker.notifyAll();
break;
+ case MERGE:
+ lsmIndex.getIOOperationCallback().beforeOperation(LSMOperationType.MERGE);
default:
break;
}
@@ -254,9 +257,7 @@
public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
throws HyracksDataException {
if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
- callback.beforeOperation();
- callback.afterOperation(null, null);
- callback.afterFinalize(null);
+ callback.afterFinalize(LSMOperationType.FLUSH, null);
return;
}
lsmIndex.scheduleFlush(ctx, callback);
@@ -271,13 +272,12 @@
ILSMComponent newComponent = null;
try {
- operation.getCallback().beforeOperation();
newComponent = lsmIndex.flush(operation);
- operation.getCallback().afterOperation(null, newComponent);
+ operation.getCallback().afterOperation(LSMOperationType.FLUSH, null, newComponent);
lsmIndex.markAsValid(newComponent);
} finally {
exitComponents(ctx, LSMOperationType.FLUSH, newComponent, false);
- operation.getCallback().afterFinalize(newComponent);
+ operation.getCallback().afterFinalize(LSMOperationType.FLUSH, newComponent);
}
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Finished the flush operation for index: " + lsmIndex);
@@ -289,9 +289,7 @@
throws HyracksDataException, IndexException {
// Merge should always be a try operation, because it should never fail to enter the components unless the merge policy is erroneous.
if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
- callback.beforeOperation();
- callback.afterOperation(null, null);
- callback.afterFinalize(null);
+ callback.afterFinalize(LSMOperationType.MERGE, null);
return;
}
lsmIndex.scheduleMerge(ctx, callback);
@@ -306,13 +304,12 @@
ILSMComponent newComponent = null;
try {
- operation.getCallback().beforeOperation();
newComponent = lsmIndex.merge(operation);
- operation.getCallback().afterOperation(ctx.getComponentHolder(), newComponent);
+ operation.getCallback().afterOperation(LSMOperationType.MERGE, ctx.getComponentHolder(), newComponent);
lsmIndex.markAsValid(newComponent);
} finally {
exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
- operation.getCallback().afterFinalize(newComponent);
+ operation.getCallback().afterFinalize(LSMOperationType.MERGE, newComponent);
}
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Finished the merge operation for index: " + lsmIndex);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
index a43b44b..2c8369e 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
@@ -19,25 +19,26 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-public enum NoOpIOOperationCallback implements ILSMIOOperationCallback, ILSMIOOperationCallbackProvider {
+public enum NoOpIOOperationCallback implements ILSMIOOperationCallback, ILSMIOOperationCallbackProvider, ILSMIOOperationCallbackFactory {
INSTANCE;
@Override
- public void beforeOperation() throws HyracksDataException {
+ public void beforeOperation(LSMOperationType opType) throws HyracksDataException {
// Do nothing.
}
@Override
- public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
// Do nothing.
}
@Override
- public void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
+ public void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) throws HyracksDataException {
// Do nothing.
}
@@ -45,4 +46,14 @@
public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
return INSTANCE;
}
+
+ @Override
+ public ILSMIOOperationCallback createIOOperationCallback() {
+ return INSTANCE;
+ }
+
+ @Override
+ public void setNumOfMutableComponents(int count) {
+ // Do nothing.
+ }
}
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 ee8febe..ed2f8cf 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
@@ -21,7 +21,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -38,17 +38,17 @@
public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ opTrackerFactory, ioScheduler, ioOpCallbackFactory);
}
public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider);
+ ioScheduler, ioOpCallbackFactory);
}
@Override
@@ -62,7 +62,8 @@
invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
+ opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+ ioOpCallbackFactory.createIOOperationCallback());
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/LSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
index 85aa312..84c7150 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -31,10 +31,10 @@
public LSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerProvider,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
double bloomFilterFalsePositiveRate) {
super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerProvider, ioSchedulerProvider,
- ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+ ioOpCallbackFactory, bloomFilterFalsePositiveRate);
}
@Override
@@ -43,7 +43,7 @@
return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
- ioOpCallbackProvider);
+ ioOpCallbackFactory);
}
}
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 1acb437..57a41ca 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
@@ -21,7 +21,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -38,17 +38,17 @@
public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, List<IVirtualBufferCache> virtualBufferCache, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
- opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ opTrackerFactory, ioScheduler, ioOpCallbackFactory);
}
public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider);
+ ioScheduler, ioOpCallbackFactory);
}
@Override
@@ -62,7 +62,8 @@
invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
- opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
+ opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+ ioOpCallbackFactory.createIOOperationCallback());
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/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
index 63d99aa..d78ae7e 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -31,10 +31,10 @@
public PartitionedLSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerProvider,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
double bloomFilterFalsePositiveRate) {
super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerProvider, ioSchedulerProvider,
- ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+ ioOpCallbackFactory, bloomFilterFalsePositiveRate);
}
@Override
@@ -43,6 +43,6 @@
return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
- ioOpCallbackProvider);
+ ioOpCallbackFactory);
}
}
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 f2f058a..cef6aee 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
@@ -52,7 +52,6 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
@@ -100,9 +99,9 @@
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ ILSMIOOperationCallback ioOpCallback) throws IndexException {
super(virtualBufferCaches, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback);
this.tokenizerFactory = tokenizerFactory;
this.invListTypeTraits = invListTypeTraits;
@@ -211,7 +210,7 @@
isActivated = false;
if (flushOnExit) {
BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
- ioOpCallbackProvider.getIOOperationCallback(this));
+ ioOpCallback);
ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
accessor.scheduleFlush(cb);
try {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index 5b0c48d..ddb6060 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -24,7 +24,7 @@
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
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.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -46,11 +46,10 @@
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ ILSMIOOperationCallback ioOpCallback) throws IndexException {
super(virtualBufferCaches, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackProvider);
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback);
}
@Override
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 ed02ba1..372221e 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
@@ -33,7 +33,7 @@
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -122,7 +122,7 @@
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+ ILSMIOOperationCallback ioOpCallback) throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -147,7 +147,7 @@
LSMInvertedIndex invIndex = new LSMInvertedIndex(virtualBufferCaches, invIndexFactory, deletedKeysBTreeFactory,
bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackProvider);
+ ioScheduler, ioOpCallback);
return invIndex;
}
@@ -157,8 +157,7 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, String onDiskDir,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
- throws IndexException {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -183,7 +182,7 @@
PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(virtualBufferCaches, invIndexFactory,
deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
- tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback);
return invIndex;
}
}
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 5d57aae..636f0cb 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
@@ -27,7 +27,7 @@
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.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -49,11 +49,11 @@
List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILinearizeComparatorFactory linearizeCmpFactory) {
this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE,
btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+ ioScheduler, ioOpCallbackFactory, linearizeCmpFactory);
}
public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
@@ -61,10 +61,10 @@
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILinearizeComparatorFactory linearizeCmpFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider);
+ ioScheduler, ioOpCallbackFactory);
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
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 bc4bca3..92364fa 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
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -44,10 +44,10 @@
List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILinearizeComparatorFactory linearizeCmpFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, btreeComparatorFactories, valueProviderFactories,
- rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+ rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackFactory, linearizeCmpFactory);
}
public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
@@ -55,11 +55,11 @@
IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILinearizeComparatorFactory linearizeCmpFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, btreeComparatorFactories,
valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider, linearizeCmpFactory);
+ ioOpCallbackFactory, linearizeCmpFactory);
}
@Override
@@ -72,8 +72,8 @@
try {
return LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
- linearizeCmpFactory);
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackFactory.createIOOperationCallback(), 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/LSMRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
index 99b4219..8254689 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -21,7 +21,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -42,10 +42,10 @@
RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyProvider mergePolicyProvider,
ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
double bloomFilterFalsePositiveRate) {
super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerFactory, ioSchedulerProvider,
- ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+ ioOpCallbackFactory, bloomFilterFalsePositiveRate);
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -59,6 +59,6 @@
virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
- ioOpCallbackProvider, linearizeCmpFactory);
+ ioOpCallbackFactory, linearizeCmpFactory);
}
}
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 72f3480..f430456 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
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -40,13 +40,14 @@
public class LSMRTreeWithAntiMatterTuplesDataflowHelper extends AbstractLSMRTreeDataflowHelper {
public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
+ int partition, List<IVirtualBufferCache> virtualBufferCaches,
+ IBinaryComparatorFactory[] btreeComparatorFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILinearizeComparatorFactory linearizeCmpFactory) {
super(opDesc, ctx, partition, virtualBufferCaches, btreeComparatorFactories, valueProviderFactories,
- rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+ rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackFactory, linearizeCmpFactory);
}
@Override
@@ -59,7 +60,8 @@
try {
return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCaches, file, diskBufferCache,
diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
- rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+ rtreePolicyType, mergePolicy, opTracker, ioScheduler,
+ ioOpCallbackFactory.createIOOperationCallback(), 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/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
index 00268b3..ef34876 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
@@ -22,7 +22,7 @@
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -40,14 +40,14 @@
private final ILSMMergePolicyProvider mergePolicyProvider;
private final ILSMOperationTrackerProvider opTrackerProvider;
private final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
- private final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
private final ILinearizeComparatorFactory linearizeCmpFactory;
public LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyProvider mergePolicyProvider,
ILSMOperationTrackerProvider opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory) {
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory) {
this.virtualBufferCacheProvider = virtualBufferCacheProvider;
this.btreeComparatorFactories = btreeComparatorFactories;
this.valueProviderFactories = valueProviderFactories;
@@ -55,7 +55,7 @@
this.mergePolicyProvider = mergePolicyProvider;
this.ioSchedulerProvider = ioSchedulerProvider;
this.opTrackerProvider = opTrackerProvider;
- this.ioOpCallbackProvider = ioOpCallbackProvider;
+ this.ioOpCallbackFactory = ioOpCallbackFactory;
this.linearizeCmpFactory = linearizeCmpFactory;
}
@@ -65,6 +65,6 @@
return new LSMRTreeWithAntiMatterTuplesDataflowHelper(opDesc, ctx, partition,
virtualBufferCacheProvider.getVirtualBufferCaches(ctx), btreeComparatorFactories,
valueProviderFactories, rtreePolicyType, mergePolicyProvider.getMergePolicy(ctx), opTrackerProvider,
- ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider, linearizeCmpFactory);
+ ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackFactory, linearizeCmpFactory);
}
}
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 04118c8..b09e115 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
@@ -40,7 +40,7 @@
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
@@ -87,9 +87,9 @@
IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallback ioOpCallback) {
super(virtualBufferCaches, componentFactory.getBufferCache(), fileManager, diskFileMapProvider,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback);
int i = 0;
for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
RTree memRTree = new RTree(virtualBufferCache,
@@ -156,7 +156,7 @@
if (flushOnExit) {
BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
- ioOpCallbackProvider.getIOOperationCallback(this));
+ ioOpCallback);
ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
accessor.scheduleFlush(cb);
try {
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 410fde8..a4b6139 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
@@ -74,12 +74,12 @@
IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallback ioOpCallback) {
super(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
btreeLeafFrameFactory, fileNameManager, new LSMRTreeDiskComponentFactory(diskRTreeFactory,
diskBTreeFactory, bloomFilterFactory), diskFileMapProvider, fieldCount, rtreeCmpFactories,
btreeCmpFactories, linearizer, comparatorFields, linearizerArray, bloomFilterFalsePositiveRate,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ mergePolicy, opTracker, ioScheduler, ioOpCallback);
}
/**
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 0996257..812e942 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
@@ -74,11 +74,11 @@
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+ ILSMIOOperationCallback ioOpCallback) {
super(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
btreeLeafFrameFactory, fileManager, new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(diskRTreeFactory),
diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
- linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallback);
bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(bulkLoadRTreeFactory);
this.bTreeTupleSorter = null;
}
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 b16d2b6..ee942e7 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
@@ -33,7 +33,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -65,7 +65,7 @@
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
ILinearizeComparatorFactory linearizeCmpFactory) throws TreeIndexException {
LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
@@ -105,7 +105,7 @@
btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ mergePolicy, opTracker, ioScheduler, ioOpCallback);
return lsmTree;
}
@@ -115,7 +115,7 @@
IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory)
+ ILSMIOOperationCallback ioOpCallback, ILinearizeComparatorFactory linearizerCmpFactory)
throws TreeIndexException {
LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
@@ -157,7 +157,7 @@
rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
rtreeCmpFactories, btreeCmpFactories, linearizerCmpFactory, comparatorFields, linearizerArray,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+ mergePolicy, opTracker, ioScheduler, ioOpCallback);
return lsmTree;
}
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 753be5d..447c507 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
@@ -54,7 +54,7 @@
return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 646343a..0303dbd 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
@@ -54,7 +54,7 @@
return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 b0617a1..dc55f5d 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
@@ -37,7 +37,7 @@
return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 bce69a2..4fc3960 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
@@ -54,7 +54,7 @@
return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 0f903e5..f553b72 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
@@ -56,7 +56,7 @@
testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
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 ab46f3f..f5cfb32 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
@@ -53,7 +53,7 @@
return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
}
@Override
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 cc3389f..2234075 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
@@ -48,7 +48,7 @@
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallback());
}
@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 d7d29a8..277df6b 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
@@ -55,7 +55,7 @@
return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
}
@Override
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 ce9ae0e..f1a5355 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
@@ -55,7 +55,7 @@
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallback());
}
@Override
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 6f8ef84..a5097b4 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
@@ -54,7 +54,7 @@
return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
- harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 1ae201f..adee618 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
@@ -53,7 +53,7 @@
return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 39e7184..6f7c23a 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
@@ -28,7 +28,7 @@
import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -66,8 +66,8 @@
public static LSMBTreeTestContext create(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
- ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback)
+ throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
int[] bloomFilterKeyFields = new int[numKeyFields];
@@ -76,7 +76,7 @@
}
LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackProvider);
+ ioScheduler, ioOpCallback);
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 216861f..b88da7b 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
@@ -32,7 +32,7 @@
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -73,7 +73,7 @@
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMMergePolicy mergePolicy;
protected ILSMOperationTracker opTracker;
- protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ protected ILSMIOOperationCallback ioOpCallback;
protected final Random rnd = new Random();
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
@@ -92,7 +92,7 @@
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
this.opTracker = new ThreadCountingTracker();
- this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+ this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
}
@@ -206,7 +206,7 @@
return mergePolicy;
}
- public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
- return ioOpCallbackProvider;
+ public ILSMIOOperationCallback getIOOperationCallback() {
+ return ioOpCallback;
}
}
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 48176b7..0907fc0 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
@@ -30,7 +30,7 @@
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -69,7 +69,7 @@
protected ILSMIOOperationScheduler ioScheduler;
protected ILSMMergePolicy mergePolicy;
protected ILSMOperationTracker opTracker;
- protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ protected ILSMIOOperationCallback ioOpCallback;
protected final Random rnd = new Random();
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
@@ -90,7 +90,7 @@
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
this.opTracker = new ThreadCountingTracker();
- this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+ this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
this.numMutableComponents = AccessMethodTestsConfig.LSM_INVINDEX_NUM_MUTABLE_COMPONENTS;
}
@@ -212,7 +212,7 @@
return mergePolicy;
}
- public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
- return ioOpCallbackProvider;
+ public ILSMIOOperationCallback getIOOperationCallback() {
+ return ioOpCallback;
}
}
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 5edf74e..059df73 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
@@ -152,8 +152,7 @@
harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
break;
}
case PARTITIONED_LSM: {
@@ -161,8 +160,7 @@
harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
break;
}
default: {
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 09daaab..dadd08c 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
@@ -57,7 +57,7 @@
return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 f5b4ae4..0b92cd3 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
@@ -57,7 +57,7 @@
return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 c94bb48..b04ae16 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
@@ -41,7 +41,7 @@
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(),
+ harness.getIOOperationCallback(),
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/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 858f90a..35c6a6b 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
@@ -57,7 +57,7 @@
return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 61692b8..6ef2e94 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
@@ -66,7 +66,7 @@
testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
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 436f9b1..94528e7 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
@@ -56,7 +56,7 @@
return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@Override
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 97a3dcc..22a6eb8 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
@@ -57,7 +57,7 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 843dcc7..10aec7e 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
@@ -57,7 +57,7 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 6ffa59a..a51abef 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
@@ -41,7 +41,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(),
+ harness.getIOOperationCallback(),
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/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 1e1f46d..e5b33cc 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
@@ -57,7 +57,7 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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 607e980..10ae5e1 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
@@ -66,7 +66,7 @@
testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
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 bdd80aa..21d3759 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
@@ -56,7 +56,7 @@
return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+ harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
}
@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/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 0c0d804..077bd45 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
@@ -61,7 +61,7 @@
harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(),
+ harness.getIOOperationCallback(),
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/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 8b54373..ef871dd 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
@@ -61,7 +61,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider(),
+ harness.getIOOperationCallback(),
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/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 9536e96..42e793b 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
@@ -26,7 +26,7 @@
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;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -71,8 +71,7 @@
IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
- throws Exception {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -80,7 +79,7 @@
fieldSerdes.length);
LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
- bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback,
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 b67274e..66e79a2 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
@@ -31,7 +31,7 @@
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -68,7 +68,7 @@
protected List<IVirtualBufferCache> virtualBufferCaches;
protected IHyracksTaskContext ctx;
protected ILSMIOOperationScheduler ioScheduler;
- protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+ protected ILSMIOOperationCallback ioOpCallback;
protected ILSMMergePolicy mergePolicy;
protected ILSMOperationTracker opTracker;
@@ -89,7 +89,7 @@
this.ioScheduler = SynchronousScheduler.INSTANCE;
this.mergePolicy = NoMergePolicy.INSTANCE;
this.opTracker = new ThreadCountingTracker();
- this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+ this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
this.numMutableComponents = AccessMethodTestsConfig.LSM_RTREE_NUM_MUTABLE_COMPONENTS;
}
@@ -206,7 +206,7 @@
return mergePolicy;
}
- public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
- return ioOpCallbackProvider;
+ public ILSMIOOperationCallback getIOOperationCallback() {
+ return ioOpCallback;
}
}
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 069524c..6a0a6bb 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
@@ -26,7 +26,7 @@
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;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -72,8 +72,8 @@
FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
- ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback)
+ throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
.serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -81,7 +81,7 @@
fieldSerdes.length);
LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCaches,
file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
- valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+ valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallback,
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
lsmTree);
diff --git a/pregelix/pregelix-benchmark/pom.xml b/pregelix/pregelix-benchmark/pom.xml
index 4d7d456..c1344ea 100644
--- a/pregelix/pregelix-benchmark/pom.xml
+++ b/pregelix/pregelix-benchmark/pom.xml
@@ -12,6 +12,42 @@
<artifactId>pregelix-benchmark</artifactId>
<name>pregelix-benchmark</name>
<url>http://maven.apache.org</url>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
+ </configuration>
+ </plugin>
+
+ <plugin>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <version>2.2-beta-5</version>
+ <configuration>
+ <descriptorRefs>
+ <descriptorRef>jar-with-dependencies</descriptorRef>
+ </descriptorRefs>
+ </configuration>
+ <executions>
+ <execution>
+ <id>make-my-jar-with-dependencies</id>
+ <phase>package</phase>
+ <goals>
+ <goal>single</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+
<dependencies>
<dependency>
<groupId>junit</groupId>
@@ -26,6 +62,12 @@
<scope>compile</scope>
</dependency>
<dependency>
+ <groupId>org.apache.giraph</groupId>
+ <artifactId>giraph-examples</artifactId>
+ <version>1.0.0</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-hdfs-core</artifactId>
<version>0.2.10-SNAPSHOT</version>
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
deleted file mode 100644
index 04c29de..0000000
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
+++ /dev/null
@@ -1,58 +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.pregelix.benchmark;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.VLongWritable;
-
-/**
- * Demonstrates the basic Pregel PageRank implementation.
- */
-public class PageRankVertex extends Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
-
- public static final String ITERATIONS = "HyracksPageRankVertex.iteration";
- private final DoubleWritable vertexValue = new DoubleWritable();
- private final DoubleWritable msg = new DoubleWritable();
- private int maxIteration = -1;
-
- @Override
- public void compute(Iterable<DoubleWritable> msgIterator) {
- if (maxIteration < 0) {
- maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
- }
- if (getSuperstep() == 1) {
- vertexValue.set(1.0 / getTotalNumVertices());
- }
- if (getSuperstep() >= 2 && getSuperstep() <= maxIteration) {
- double sum = 0;
- for (DoubleWritable msg : msgIterator) {
- sum += msg.get();
- }
- vertexValue.set((0.15 / getTotalNumVertices()) + 0.85 * sum);
- }
-
- if (getSuperstep() >= 1 && getSuperstep() < maxIteration) {
- long edges = getNumEdges();
- msg.set(vertexValue.get() / edges);
- sendMessageToAllEdges(msg);
- } else {
- voteToHalt();
- }
- }
-
-}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
similarity index 60%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
index 3d85f66..b290907 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
@@ -12,7 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io;
import java.io.IOException;
import java.util.ArrayList;
@@ -24,14 +25,13 @@
import org.apache.giraph.edge.Edge;
import org.apache.giraph.edge.MapMutableEdge;
import org.apache.giraph.io.formats.TextVertexInputFormat;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextCCInputFormat extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
@Override
public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,25 +39,25 @@
String[] items;
@Override
- protected VLongWritable getId(Text line) throws IOException {
+ protected LongWritable getId(Text line) throws IOException {
items = line.toString().split(" ");
- return new VLongWritable(Long.parseLong(items[0]));
+ return new LongWritable(Long.parseLong(items[0]));
}
@Override
- protected DoubleWritable getValue(Text line) throws IOException {
+ protected LongWritable getValue(Text line) throws IOException {
return null;
}
@Override
- protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
- List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
- Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+ protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+ List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+ Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
for (int i = 1; i < items.length; i++) {
- edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+ edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
}
- for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
- MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+ for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
edge.setEntry(entry);
edge.setValue(null);
edges.add(edge);
@@ -67,4 +67,5 @@
};
}
+
}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
new file mode 100644
index 0000000..770c6e1
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * 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.benchmark.io;
+
+import java.io.IOException;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextCCOutputFormat extends TextVertexOutputFormat<LongWritable, LongWritable, NullWritable> {
+
+ @Override
+ public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+ return new TextVertexWriterToEachLine() {
+
+ @Override
+ protected Text convertVertexToLine(Vertex<LongWritable, LongWritable, NullWritable, ?> vertex)
+ throws IOException {
+ return new Text(vertex.getId() + " " + vertex.getValue());
+ }
+
+ };
+ }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
similarity index 64%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
index 3d85f66..38eef3a 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.benchmark;
+package edu.uci.ics.pregelix.benchmark.io;
import java.io.IOException;
import java.util.ArrayList;
@@ -25,13 +25,13 @@
import org.apache.giraph.edge.MapMutableEdge;
import org.apache.giraph.io.formats.TextVertexInputFormat;
import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextPRInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
@Override
public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,9 @@
String[] items;
@Override
- protected VLongWritable getId(Text line) throws IOException {
+ protected LongWritable getId(Text line) throws IOException {
items = line.toString().split(" ");
- return new VLongWritable(Long.parseLong(items[0]));
+ return new LongWritable(Long.parseLong(items[0]));
}
@Override
@@ -50,14 +50,14 @@
}
@Override
- protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
- List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
- Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+ protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+ List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+ Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
for (int i = 1; i < items.length; i++) {
- edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+ edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
}
- for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
- MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+ for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
edge.setEntry(entry);
edge.setValue(null);
edges.add(edge);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java
new file mode 100644
index 0000000..b14de6f
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java
@@ -0,0 +1,43 @@
+/*
+ * 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.benchmark.io;
+
+import java.io.IOException;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextPROutputFormat extends TextVertexOutputFormat<LongWritable, DoubleWritable, NullWritable> {
+
+ @Override
+ public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+ return new TextVertexWriterToEachLine() {
+
+ @Override
+ protected Text convertVertexToLine(Vertex<LongWritable, DoubleWritable, NullWritable, ?> vertex)
+ throws IOException {
+ return new Text(vertex.getId() + " " + vertex.getValue());
+ }
+
+ };
+ }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
similarity index 65%
rename from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
rename to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
index 3d85f66..953e93c 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
@@ -12,7 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io;
import java.io.IOException;
import java.util.ArrayList;
@@ -25,13 +26,12 @@
import org.apache.giraph.edge.MapMutableEdge;
import org.apache.giraph.io.formats.TextVertexInputFormat;
import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextSPInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
@Override
public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,9 @@
String[] items;
@Override
- protected VLongWritable getId(Text line) throws IOException {
+ protected LongWritable getId(Text line) throws IOException {
items = line.toString().split(" ");
- return new VLongWritable(Long.parseLong(items[0]));
+ return new LongWritable(Long.parseLong(items[0]));
}
@Override
@@ -50,16 +50,16 @@
}
@Override
- protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
- List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
- Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+ protected Iterable<Edge<LongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+ List<Edge<LongWritable, DoubleWritable>> edges = new ArrayList<Edge<LongWritable, DoubleWritable>>();
+ Map<LongWritable, DoubleWritable> edgeMap = new HashMap<LongWritable, DoubleWritable>();
for (int i = 1; i < items.length; i++) {
- edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+ edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
}
- for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
- MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+ for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
edge.setEntry(entry);
- edge.setValue(null);
+ edge.setValue(new DoubleWritable(1.0));
edges.add(edge);
}
return edges;
@@ -67,4 +67,5 @@
};
}
+
}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
new file mode 100644
index 0000000..0a70b3c
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
@@ -0,0 +1,72 @@
+/*
+ * 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.benchmark.io2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.MapMutableEdge;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextCCInputFormat2 extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
+
+ @Override
+ public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
+ return new TextVertexReaderFromEachLine() {
+ String[] items;
+
+ @Override
+ protected LongWritable getId(Text line) throws IOException {
+ String[] kv = line.toString().split("\t");
+ items = kv[1].split(" ");
+ return new LongWritable(Long.parseLong(kv[0]));
+ }
+
+ @Override
+ protected LongWritable getValue(Text line) throws IOException {
+ return null;
+ }
+
+ @Override
+ protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+ List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+ Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
+ for (int i = 1; i < items.length; i++) {
+ edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
+ }
+ for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
+ edge.setEntry(entry);
+ edge.setValue(null);
+ edges.add(edge);
+ }
+ return edges;
+ }
+
+ };
+ }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
similarity index 62%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
index 3d85f66..63a4519 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.benchmark;
+package edu.uci.ics.pregelix.benchmark.io2;
import java.io.IOException;
import java.util.ArrayList;
@@ -25,13 +25,13 @@
import org.apache.giraph.edge.MapMutableEdge;
import org.apache.giraph.io.formats.TextVertexInputFormat;
import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextPRInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
@Override
public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,10 @@
String[] items;
@Override
- protected VLongWritable getId(Text line) throws IOException {
- items = line.toString().split(" ");
- return new VLongWritable(Long.parseLong(items[0]));
+ protected LongWritable getId(Text line) throws IOException {
+ String[] kv = line.toString().split("\t");
+ items = kv[1].split(" ");
+ return new LongWritable(Long.parseLong(items[0]));
}
@Override
@@ -50,14 +51,14 @@
}
@Override
- protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
- List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
- Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+ protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+ List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+ Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
for (int i = 1; i < items.length; i++) {
- edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+ edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
}
- for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
- MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+ for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
edge.setEntry(entry);
edge.setValue(null);
edges.add(edge);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
similarity index 63%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
index 3d85f66..fdb1061 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
@@ -12,7 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io2;
import java.io.IOException;
import java.util.ArrayList;
@@ -25,13 +26,12 @@
import org.apache.giraph.edge.MapMutableEdge;
import org.apache.giraph.io.formats.TextVertexInputFormat;
import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextSPInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
@Override
public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,10 @@
String[] items;
@Override
- protected VLongWritable getId(Text line) throws IOException {
- items = line.toString().split(" ");
- return new VLongWritable(Long.parseLong(items[0]));
+ protected LongWritable getId(Text line) throws IOException {
+ String[] kv = line.toString().split("\t");
+ items = kv[1].split(" ");
+ return new LongWritable(Long.parseLong(kv[0]));
}
@Override
@@ -50,16 +51,16 @@
}
@Override
- protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
- List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
- Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+ protected Iterable<Edge<LongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+ List<Edge<LongWritable, DoubleWritable>> edges = new ArrayList<Edge<LongWritable, DoubleWritable>>();
+ Map<LongWritable, DoubleWritable> edgeMap = new HashMap<LongWritable, DoubleWritable>();
for (int i = 1; i < items.length; i++) {
- edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+ edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
}
- for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
- MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+ for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
edge.setEntry(entry);
- edge.setValue(null);
+ edge.setValue(new DoubleWritable(1.0));
edges.add(edge);
}
return edges;
@@ -67,4 +68,5 @@
};
}
+
}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java
new file mode 100644
index 0000000..3789d6d
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java
@@ -0,0 +1,78 @@
+/*
+ * 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.benchmark.vertex;
+
+import java.io.IOException;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+public class ConnectedComponentsVertex extends Vertex<LongWritable, LongWritable, NullWritable, LongWritable> {
+ /**
+ * Propagates the smallest vertex id to all neighbors. Will always choose to
+ * halt and only reactivate if a smaller id has been sent to it.
+ *
+ * @param messages
+ * Iterator of messages from the previous superstep.
+ * @throws IOException
+ */
+ @Override
+ public void compute(Iterable<LongWritable> messages) throws IOException {
+ long currentComponent = getValue().get();
+
+ // First superstep is special, because we can simply look at the neighbors
+ if (getSuperstep() == 0) {
+ for (Edge<LongWritable, NullWritable> edge : getEdges()) {
+ long neighbor = edge.getTargetVertexId().get();
+ if (neighbor < currentComponent) {
+ currentComponent = neighbor;
+ }
+ }
+ // Only need to send value if it is not the own id
+ if (currentComponent != getValue().get()) {
+ setValue(new LongWritable(currentComponent));
+ for (Edge<LongWritable, NullWritable> edge : getEdges()) {
+ LongWritable neighbor = edge.getTargetVertexId();
+ if (neighbor.get() > currentComponent) {
+ sendMessage(neighbor, getValue());
+ }
+ }
+ }
+
+ voteToHalt();
+ return;
+ }
+
+ boolean changed = false;
+ // did we get a smaller id ?
+ for (LongWritable message : messages) {
+ long candidateComponent = message.get();
+ if (candidateComponent < currentComponent) {
+ currentComponent = candidateComponent;
+ changed = true;
+ }
+ }
+
+ // propagate new component id to the neighbors
+ if (changed) {
+ setValue(new LongWritable(currentComponent));
+ sendMessageToAllEdges(getValue());
+ }
+ voteToHalt();
+ }
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java
new file mode 100644
index 0000000..86e90dd
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java
@@ -0,0 +1,48 @@
+/*
+ * 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.benchmark.vertex;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.examples.RandomWalkVertex;
+import org.apache.giraph.utils.MathUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * The PageRank algorithm, with uniform transition probabilities on the edges
+ * http://en.wikipedia.org/wiki/PageRank
+ */
+public class PageRankVertex extends RandomWalkVertex<NullWritable> {
+
+ @Override
+ protected double transitionProbability(double stateProbability, Edge<LongWritable, NullWritable> edge) {
+ return stateProbability / getNumEdges();
+ }
+
+ @Override
+ protected double recompute(Iterable<DoubleWritable> partialRanks, double teleportationProbability) {
+
+ // rank contribution from incident neighbors
+ double rankFromNeighbors = MathUtils.sum(partialRanks);
+ // rank contribution from dangling vertices
+ double danglingContribution = getDanglingProbability() / getTotalNumVertices();
+
+ // recompute rank
+ return (1d - teleportationProbability) * (rankFromNeighbors + danglingContribution) + teleportationProbability
+ / getTotalNumVertices();
+ }
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java
new file mode 100644
index 0000000..755a3d0
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.benchmark.vertex;
+
+import java.io.IOException;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Shortest paths algorithm.
+ */
+public class ShortestPathsVertex extends Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> {
+ /** Source id. */
+ public static final String SOURCE_ID = "giraph.shortestPathsBenchmark.sourceId";
+ /** Default source id. */
+ public static final long SOURCE_ID_DEFAULT = 1;
+
+ private boolean isSource() {
+ return getId().get() == getConf().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+ }
+
+ @Override
+ public void compute(Iterable<DoubleWritable> messages) throws IOException {
+ if (getSuperstep() == 0) {
+ setValue(new DoubleWritable(Double.MAX_VALUE));
+ }
+
+ double minDist = isSource() ? 0d : Double.MAX_VALUE;
+ for (DoubleWritable message : messages) {
+ minDist = Math.min(minDist, message.get());
+ }
+
+ if (minDist < getValue().get()) {
+ setValue(new DoubleWritable(minDist));
+ for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
+ double distance = minDist + edge.getValue().get();
+ sendMessage(edge.getTargetVertexId(), new DoubleWritable(distance));
+ }
+ }
+
+ voteToHalt();
+ }
+}
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 4863378..88edc91 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
@@ -63,6 +63,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
@@ -95,7 +96,8 @@
import edu.uci.ics.pregelix.core.data.TypeTraits;
import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawNormalizedKeyComputerFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
import edu.uci.ics.pregelix.dataflow.ClearStateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -196,17 +198,14 @@
return jobId;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
public JobSpecification generateCreatingJob() throws HyracksException {
- Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
JobSpecification spec = new JobSpecification();
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
typeTraits[1] = new TypeTraits(false);
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
int[] keyFields = new int[1];
keyFields[0] = 0;
@@ -264,8 +263,7 @@
sortFields[0] = 0;
INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameLimit, sortFields,
nkmFactory, comparatorFactories, recordDescriptor);
ClusterConfig.setLocationConstraint(spec, sorter);
@@ -291,12 +289,12 @@
ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
- comparatorFactories), sorter, 0, writer, 0);
+ comparatorFactories, nkmFactory), sorter, 0, writer, 0);
spec.setFrameSize(frameSize);
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
+ @SuppressWarnings({ "rawtypes" })
public JobSpecification scanIndexPrintGraph(String nodeName, String path) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
@@ -324,8 +322,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
@@ -355,16 +352,17 @@
*/
int[] sortFields = new int[1];
sortFields[0] = 0;
+ INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
- comparatorFactories), scanner, 0, writer, 0);
+ comparatorFactories, nkmFactory), scanner, 0, writer, 0);
spec.setFrameSize(frameSize);
return spec;
}
public JobSpecification scanIndexWriteGraph() throws HyracksException {
- JobSpecification spec = scanIndexWriteToHDFS(conf);
+ JobSpecification spec = scanIndexWriteToHDFS(conf, false);
return spec;
}
@@ -380,7 +378,7 @@
FileSystem dfs = FileSystem.get(tmpJob.getConfiguration());
dfs.delete(new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)), true);
- JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration());
+ JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration(), true);
dfs.delete(new Path(BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration)), true);
JobSpecification[] stateCkpSpecs = generateStateCheckpointing(lastSuccessfulIteration);
@@ -523,10 +521,9 @@
*/
int[] sortFields = new int[1];
sortFields[0] = 0;
- INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+ INormalizedKeyComputerFactory nkmFactory = RawNormalizedKeyComputerFactory.INSTANCE;
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, sortFields,
nkmFactory, comparatorFactories, recordDescriptor);
ClusterConfig.setLocationConstraint(spec, sorter);
@@ -556,8 +553,9 @@
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
- private JobSpecification scanIndexWriteToHDFS(Configuration conf) throws HyracksDataException, HyracksException {
+ @SuppressWarnings({ "rawtypes" })
+ private JobSpecification scanIndexWriteToHDFS(Configuration conf, boolean ckpointing) throws HyracksDataException,
+ HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
JobSpecification spec = new JobSpecification();
@@ -584,8 +582,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -597,6 +594,18 @@
null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, scanner);
+ ExternalSortOperatorDescriptor sort = null;
+ if (!ckpointing) {
+ int[] keyFields = new int[] { 0 };
+ INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getFinalNormalizedKeyComputerFactory(conf);
+ IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+ sortCmpFactories[0] = JobGenUtil.getFinalBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
+ .getClass());
+ sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields, nkmFactory, sortCmpFactories,
+ recordDescriptor);
+ ClusterConfig.setLocationConstraint(spec, scanner);
+ }
+
/**
* construct write file operator
*/
@@ -610,7 +619,12 @@
* connect operator descriptors
*/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+ if (!ckpointing) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sort, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, writer, 0);
+ } else {
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+ }
spec.setFrameSize(frameSize);
return spec;
}
@@ -715,7 +729,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastCheckpointedIteration,
WritableComparator.get(vertexIdClass).getClass());
ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, recordDescriptor);
+ nkmFactory, sortCmpFactories, recordDescriptor, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, sort);
/**
@@ -739,8 +753,7 @@
* connect operator descriptors
*/
ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
- sort, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sort, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
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 9389f62..103c1b6 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
@@ -50,6 +50,7 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -62,7 +63,7 @@
import edu.uci.ics.pregelix.core.data.TypeTraits;
import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -97,7 +98,6 @@
super(job);
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
@@ -129,8 +129,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
@@ -195,7 +194,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
@@ -282,8 +281,8 @@
spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 5, btreeBulkLoad, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
- spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
- localGby, 0, globalGby, 0);
+ spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+ nkmFactory), localGby, 0, globalGby, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
@@ -300,7 +299,6 @@
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -316,8 +314,7 @@
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -401,7 +398,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
@@ -503,8 +500,8 @@
spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 5, btreeBulkLoad, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
- spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
- localGby, 0, globalGby, 0);
+ spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+ nkmFactory), localGby, 0, globalGby, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
@@ -601,7 +598,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastSuccessfulIteration,
WritableComparator.get(vertexIdClass).getClass());
ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, recordDescriptor);
+ nkmFactory, sortCmpFactories, recordDescriptor, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, sort);
/**
@@ -633,7 +630,7 @@
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
+ @SuppressWarnings({ "rawtypes" })
private JobSpecification generateSecondaryBTreeCheckpoint(int lastSuccessfulIteration, PregelixJob job)
throws HyracksException {
job.setOutputFormatClass(SequenceFileOutputFormat.class);
@@ -668,8 +665,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), msgListClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
ITypeTraits[] typeTraits = new ITypeTraits[2];
typeTraits[0] = new TypeTraits(false);
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 287b797..308f422 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
@@ -32,6 +32,7 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -42,7 +43,7 @@
import edu.uci.ics.pregelix.core.data.TypeTraits;
import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -74,7 +75,6 @@
super(job);
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -101,8 +101,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -138,7 +137,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
@@ -241,8 +240,8 @@
* connect the group-by operator
*/
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
- spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
- localGby, 0, globalGby, 0);
+ spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+ nkmFactory), localGby, 0, globalGby, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink2, 0);
@@ -258,7 +257,6 @@
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -274,8 +272,7 @@
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -338,7 +335,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
@@ -440,8 +437,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
- spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
- localGby, 0, globalGby, 0);
+ spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+ nkmFactory), localGby, 0, globalGby, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 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 3b3c9e7..718a271 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
@@ -31,6 +31,7 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -41,7 +42,7 @@
import edu.uci.ics.pregelix.core.data.TypeTraits;
import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -73,7 +74,6 @@
super(job);
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -100,8 +100,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -140,7 +139,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, globalSort);
/**
@@ -247,7 +246,6 @@
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -263,8 +261,7 @@
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -327,7 +324,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, globalSort);
/**
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 e334095..75635c9 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
@@ -31,6 +31,7 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -41,7 +42,7 @@
import edu.uci.ics.pregelix.core.data.TypeTraits;
import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -73,7 +74,6 @@
super(job);
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -100,8 +100,7 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -137,7 +136,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
@@ -153,7 +152,7 @@
* construct global sort operator
*/
ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, globalSort);
/**
@@ -261,7 +260,6 @@
return spec;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -277,8 +275,7 @@
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
- .getClass());
+ comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -341,7 +338,7 @@
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
@@ -357,7 +354,7 @@
* construct global sort operator
*/
ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
- nkmFactory, sortCmpFactories, rdUnnestedMessage);
+ nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
ClusterConfig.setLocationConstraint(spec, globalSort);
/**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
index c4f54fe..97cea99 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
@@ -21,10 +21,12 @@
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawNormalizedKeyComputerFactory;
import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNormalizedKeyComputerFactory;
-@SuppressWarnings({ "rawtypes", "unchecked" })
+@SuppressWarnings({ "rawtypes" })
public class JobGenUtil {
/**
@@ -36,11 +38,7 @@
* @return
*/
public static INormalizedKeyComputerFactory getINormalizedKeyComputerFactory(Configuration conf) {
- Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
- if (clazz.equals(NormalizedKeyComputer.class)) {
- return null;
- }
- return new VertexIdNormalizedKeyComputerFactory(clazz);
+ return RawNormalizedKeyComputerFactory.INSTANCE;
}
/**
@@ -52,6 +50,33 @@
* @return
*/
public static IBinaryComparatorFactory getIBinaryComparatorFactory(int iteration, Class keyClass) {
+ return RawBinaryComparatorFactory.INSTANCE;
+ }
+
+ /**
+ * get normalized key factory for the final output job
+ *
+ * @param iteration
+ * @param keyClass
+ * @return
+ */
+ public static INormalizedKeyComputerFactory getFinalNormalizedKeyComputerFactory(Configuration conf) {
+ Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
+ if (clazz.equals(NormalizedKeyComputer.class)) {
+ return null;
+ }
+ return new VertexIdNormalizedKeyComputerFactory(clazz);
+ }
+
+ /**
+ * get comparator for the final output job
+ *
+ * @param iteration
+ * @param keyClass
+ * @return
+ */
+ @SuppressWarnings("unchecked")
+ public static IBinaryComparatorFactory getFinalBinaryComparatorFactory(Class keyClass) {
return new WritableComparingBinaryComparatorFactory(keyClass);
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..c0173ed
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java
@@ -0,0 +1,46 @@
+/*
+ * 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.core.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+ private RawBinaryComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ int commonLength = Math.min(l1, l2);
+ for (int i = 0; i < commonLength; i++) {
+ if (b1[s1 + i] != b2[s2 + i]) {
+ return (b1[s1 + i] & 0xff) - (b2[s2 + i] & 0xff);
+ }
+ }
+ int difference = l1 - l2;
+ return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+ }
+
+ };
+ }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..6aa4af9
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.core.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class RawNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+ private static final long serialVersionUID = 1L;
+ public static final INormalizedKeyComputerFactory INSTANCE = new RawNormalizedKeyComputerFactory();
+
+ private RawNormalizedKeyComputerFactory() {
+
+ }
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ int nk = 0;
+ for (int i = 0; i < 4; i++) {
+ nk <<= 8;
+ if (i < length) {
+ nk += (bytes[start + i] & 0xff);
+ }
+ }
+ return nk;
+ }
+
+ };
+ }
+}
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 4c7f91d..f599996 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
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -34,6 +35,7 @@
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -89,6 +91,7 @@
UTF8StringPointable.FACTORY);
private IBinaryComparatorFactory stringComparatorFactory = new PointableBinaryComparatorFactory(
UTF8StringPointable.FACTORY);
+ private INormalizedKeyComputerFactory nmkFactory = new UTF8StringNormalizedKeyComputerFactory();
private void cleanupStores() throws IOException {
FileUtils.forceMkdir(new File("teststore"));
@@ -211,7 +214,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, sorter, 0);
IConnectorDescriptor joinWriterConn = new MToNPartitioningMergingConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
- new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories);
+ new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+ nmkFactory);
spec.connect(joinWriterConn, sorter, 0, writer, 0);
spec.addRoot(writer);
@@ -284,8 +288,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorter, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- sortFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
- sorter, 0, writer, 0);
+ sortFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+ nmkFactory), sorter, 0, writer, 0);
spec.addRoot(writer);
runTest(spec);
@@ -368,11 +372,11 @@
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
- sorter, 0, join, 0);
+ keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+ nmkFactory), sorter, 0, join, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
- join, 0, writer, 0);
+ keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+ nmkFactory), join, 0, writer, 0);
spec.addRoot(writer);
runTest(spec);
@@ -477,7 +481,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), project, 0, sorter, 0);
IConnectorDescriptor joinWriterConn = new MToNPartitioningMergingConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 9 },
- new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories);
+ new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+ nmkFactory);
spec.connect(joinWriterConn, sorter, 0, writer, 0);
spec.addRoot(writer);
@@ -573,7 +578,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
keyFields, new IBinaryHashFunctionFactory[] { new PointableBinaryHashFunctionFactory(
- UTF8StringPointable.FACTORY) }), sortFields, comparatorFactories), sorter, 0, join, 0);
+ UTF8StringPointable.FACTORY) }), sortFields, comparatorFactories, nmkFactory), sorter, 0, join,
+ 0);
IBinaryComparatorFactory[] mergeComparatorFactories = new IBinaryComparatorFactory[2];
mergeComparatorFactories[0] = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY);
@@ -581,7 +587,8 @@
int[] mergeFields = new int[] { 9, 0 };
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
new int[] { 9 }, new IBinaryHashFunctionFactory[] { new PointableBinaryHashFunctionFactory(
- UTF8StringPointable.FACTORY) }), mergeFields, comparatorFactories), join, 0, writer, 0);
+ UTF8StringPointable.FACTORY) }), mergeFields, comparatorFactories, nmkFactory), join, 0,
+ writer, 0);
spec.addRoot(writer);
runTest(spec);