Merge branch 'master' into zheilbron/hyracks_asterix_issue470
Conflicts:
hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
index c4b7802..733c7d3 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
@@ -27,11 +27,9 @@
public interface ICCApplicationContext extends IApplicationContext {
/**
* Sets the state that must be distributed by the infrastructure to all the
- * NC application contexts. Any state set by calling this method in
- * the {@link ICCApplicationEntryPoint#start(ICCApplicationContext, String[])} call
- * is made available to all the {@link INCApplicationContext} objects at each Node Controller.
- * The state is then available to be inspected by the application at the NC during or
- * after the {@link INCBootstrap#start()} call.
+ * NC application contexts. Any state set by calling this method in the {@link ICCApplicationEntryPoint#start(ICCApplicationContext, String[])} call is made available to all the {@link INCApplicationContext} objects
+ * at each Node Controller. The state is then available to be inspected by
+ * the application at the NC during or after the {@link INCBootstrap#start()} call.
*
* @param state
* The distributed state
@@ -47,6 +45,14 @@
public void addJobLifecycleListener(IJobLifecycleListener jobLifecycleListener);
/**
+ * A listener that listens to Cluster Lifecycle events at the Cluster
+ * Controller.
+ *
+ * @param jobLifecycleListener
+ */
+ public void addClusterLifecycleListener(IClusterLifecycleListener clusterLifecycleListener);
+
+ /**
* Get the Cluster Controller Context.
*
* @return The Cluster Controller Context.
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IClusterLifecycleListener.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IClusterLifecycleListener.java
new file mode 100644
index 0000000..51db13e
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IClusterLifecycleListener.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.application;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A listener interface for providing notification call backs to events such as a Node Controller joining/leaving the cluster.
+ */
+public interface IClusterLifecycleListener {
+
+ /**
+ * @param nodeId
+ * A unique identifier of a Node Controller
+ * @param ncConfig
+ * A map containing the set of configuration parameters that were used to start the Node Controller
+ */
+ public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration);
+
+ /**
+ * @param deadNodeIds
+ * A set of Node Controller Ids that have left the cluster. The set is not cumulative.
+ */
+ public void notifyNodeFailure(Set<String> deadNodeIds);
+
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index 7e1581a..f884c6b 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -19,9 +19,11 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.application.IClusterLifecycleListener;
import edu.uci.ics.hyracks.api.context.ICCContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
@@ -41,6 +43,7 @@
protected IResultCallback<Object> deinitializationCallback;
private List<IJobLifecycleListener> jobLifecycleListeners;
+ private List<IClusterLifecycleListener> clusterLifecycleListeners;
public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext) throws IOException {
super(serverCtx);
@@ -48,6 +51,7 @@
initPendingNodeIds = new HashSet<String>();
deinitPendingNodeIds = new HashSet<String>();
jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
+ clusterLifecycleListeners = new ArrayList<IClusterLifecycleListener>();
}
public ICCContext getCCContext() {
@@ -82,4 +86,21 @@
l.notifyJobCreation(jobId, acggf);
}
}
+
+ @Override
+ public void addClusterLifecycleListener(IClusterLifecycleListener clusterLifecycleListener) {
+ clusterLifecycleListeners.add(clusterLifecycleListener);
+ }
+
+ public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) throws HyracksException {
+ for (IClusterLifecycleListener l : clusterLifecycleListeners) {
+ l.notifyNodeJoin(nodeId, ncConfiguration);
+ }
+ }
+
+ public void notifyNodeFailure(Set<String> deadNodeIds) {
+ for (IClusterLifecycleListener l : clusterLifecycleListeners) {
+ l.notifyNodeFailure(deadNodeIds);
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
index f7dd1d2..03d43ed 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@@ -47,6 +48,7 @@
IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
CCNCFunctions.NodeRegistrationResult result = null;
+ Map<String, String> ncConfiguration = null;
try {
INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
@@ -58,6 +60,8 @@
nodeMap.put(id, state);
Map<String, Set<String>> ipAddressNodeNameMap = ccs.getIpAddressNodeNameMap();
String ipAddress = state.getNCConfig().dataIPAddress;
+ ncConfiguration = new HashMap<String, String>();
+ state.getNCConfig().toMap(ncConfiguration);
Set<String> nodes = ipAddressNodeNameMap.get(ipAddress);
if (nodes == null) {
nodes = new HashSet<String>();
@@ -75,5 +79,6 @@
result = new CCNCFunctions.NodeRegistrationResult(null, e);
}
ncIPCHandle.send(-1, result, null);
+ ccs.getApplicationContext().notifyNodeJoin(id, ncConfiguration);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
index 7255503..c82e264 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -65,6 +65,9 @@
}
}
}
+ if (deadNodes != null && deadNodes.size() > 0) {
+ ccs.getApplicationContext().notifyNodeFailure(deadNodes);
+ }
}
@Override
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index ec29592..633d4fb 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -16,86 +16,107 @@
import java.io.Serializable;
import java.util.List;
+import java.util.Map;
import org.kohsuke.args4j.Argument;
import org.kohsuke.args4j.Option;
import org.kohsuke.args4j.spi.StopOptionHandler;
public class NCConfig implements Serializable {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- @Option(name = "-cc-host", usage = "Cluster Controller host name", required = true)
- public String ccHost;
+ @Option(name = "-cc-host", usage = "Cluster Controller host name", required = true)
+ public String ccHost;
- @Option(name = "-cc-port", usage = "Cluster Controller port (default: 1099)")
- public int ccPort = 1099;
+ @Option(name = "-cc-port", usage = "Cluster Controller port (default: 1099)")
+ public int ccPort = 1099;
- @Option(name = "-cluster-net-ip-address", usage = "IP Address to bind cluster listener", required = true)
- public String clusterNetIPAddress;
+ @Option(name = "-cluster-net-ip-address", usage = "IP Address to bind cluster listener", required = true)
+ public String clusterNetIPAddress;
- @Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster", required = true)
- public String nodeId;
+ @Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster", required = true)
+ public String nodeId;
- @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
- public String dataIPAddress;
+ @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
+ public String dataIPAddress;
- @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
- public String datasetIPAddress;
+ @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
+ public String datasetIPAddress;
- @Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
- public String ioDevices = System.getProperty("java.io.tmpdir");
+ @Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
+ public String ioDevices = System.getProperty("java.io.tmpdir");
- @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
- public int nNetThreads = 1;
+ @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
+ public int nNetThreads = 1;
- @Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
- public int maxMemory = -1;
+ @Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
+ public int maxMemory = -1;
- @Option(name = "-result-history-size", usage = "Limits the number of jobs whose results should be remembered by the system to the specified value. (default: 10)")
- public int resultHistorySize = 100;
+ @Option(name = "-result-history-size", usage = "Limits the number of jobs whose results should be remembered by the system to the specified value. (default: 10)")
+ public int resultHistorySize = 100;
- @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
- public int resultManagerMemory = -1;
+ @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
+ public int resultManagerMemory = -1;
- @Option(name = "-app-nc-main-class", usage = "Application NC Main Class")
- public String appNCMainClass;
+ @Option(name = "-app-nc-main-class", usage = "Application NC Main Class")
+ public String appNCMainClass;
- @Argument
- @Option(name = "--", handler = StopOptionHandler.class)
- public List<String> appArgs;
+ @Argument
+ @Option(name = "--", handler = StopOptionHandler.class)
+ public List<String> appArgs;
- public void toCommandLine(List<String> cList) {
- cList.add("-cc-host");
- cList.add(ccHost);
- cList.add("-cc-port");
- cList.add(String.valueOf(ccPort));
- cList.add("-cluster-net-ip-address");
- cList.add(clusterNetIPAddress);
- cList.add("-node-id");
- cList.add(nodeId);
- cList.add("-data-ip-address");
- cList.add(dataIPAddress);
- cList.add(datasetIPAddress);
- cList.add("-iodevices");
- cList.add(ioDevices);
- cList.add("-net-thread-count");
- cList.add(String.valueOf(nNetThreads));
- cList.add("-max-memory");
- cList.add(String.valueOf(maxMemory));
- cList.add("-result-history-size");
- cList.add(String.valueOf(resultHistorySize));
- cList.add("-result-manager-memory");
- cList.add(String.valueOf(resultManagerMemory));
+ public void toCommandLine(List<String> cList) {
+ cList.add("-cc-host");
+ cList.add(ccHost);
+ cList.add("-cc-port");
+ cList.add(String.valueOf(ccPort));
+ cList.add("-cluster-net-ip-address");
+ cList.add(clusterNetIPAddress);
+ cList.add("-node-id");
+ cList.add(nodeId);
+ cList.add("-data-ip-address");
+ cList.add(dataIPAddress);
+ cList.add(datasetIPAddress);
+ cList.add("-iodevices");
+ cList.add(ioDevices);
+ cList.add("-net-thread-count");
+ cList.add(String.valueOf(nNetThreads));
+ cList.add("-max-memory");
+ cList.add(String.valueOf(maxMemory));
+ cList.add("-result-history-size");
+ cList.add(String.valueOf(resultHistorySize));
+ cList.add("-result-manager-memory");
+ cList.add(String.valueOf(resultManagerMemory));
- if (appNCMainClass != null) {
- cList.add("-app-nc-main-class");
- cList.add(appNCMainClass);
- }
- if (appArgs != null && !appArgs.isEmpty()) {
- cList.add("--");
- for (String appArg : appArgs) {
- cList.add(appArg);
- }
- }
- }
-}
+ if (appNCMainClass != null) {
+ cList.add("-app-nc-main-class");
+ cList.add(appNCMainClass);
+ }
+ if (appArgs != null && !appArgs.isEmpty()) {
+ cList.add("--");
+ for (String appArg : appArgs) {
+ cList.add(appArg);
+ }
+ }
+ }
+
+ public void toMap(Map<String, String> configuration) {
+ configuration.put("cc-host", ccHost);
+ configuration.put("cc-port", (String.valueOf(ccPort)));
+ configuration.put("cluster-net-ip-address", clusterNetIPAddress);
+ configuration.put("node-id", nodeId);
+ configuration.put("data-ip-address", dataIPAddress);
+ configuration.put("iodevices", ioDevices);
+ configuration.put("net-thread-count", String.valueOf(nNetThreads));
+ configuration.put("max-memory", String.valueOf(maxMemory));
+ configuration.put("result-history-size",
+ String.valueOf(resultHistorySize));
+ configuration.put("result-manager-memory",
+ String.valueOf(resultManagerMemory));
+
+ if (appNCMainClass != null) {
+ configuration.put("app-nc-main-class", appNCMainClass);
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
index b74d97b..35649ca 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
@@ -23,17 +23,25 @@
private static final long serialVersionUID = 1L;
private final String nodeName;
-
private final FileReference file;
+ private final int ioDeviceId;
public FileSplit(String nodeName, FileReference file) {
this.nodeName = nodeName;
this.file = file;
+ this.ioDeviceId = 0;
+ }
+
+ public FileSplit(String nodeName, FileReference file, int ioDeviceId) {
+ this.nodeName = nodeName;
+ this.file = file;
+ this.ioDeviceId = ioDeviceId;
}
public FileSplit(String nodeName, String path) {
this.nodeName = nodeName;
this.file = new FileReference(new File(path));
+ this.ioDeviceId = 0;
}
public String getNodeName() {
@@ -43,4 +51,8 @@
public FileReference getLocalFile() {
return file;
}
+
+ public int getIODeviceId() {
+ return ioDeviceId;
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index bc0189e..ebf3848 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -103,25 +103,26 @@
accessorProbe.reset(buffer);
int tupleCount0 = accessorProbe.getTupleCount();
for (int i = 0; i < tupleCount0; ++i) {
- int entry = tpcProbe.partition(accessorProbe, i, tableSize);
- boolean matchFound = false;
- int offset = 0;
- do {
- table.getTuplePointer(entry, offset++, storedTuplePointer);
- if (storedTuplePointer.frameIndex < 0)
- break;
- int bIndex = storedTuplePointer.frameIndex;
- int tIndex = storedTuplePointer.tupleIndex;
- accessorBuild.reset(buffers.get(bIndex));
- int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
- if (c == 0) {
- matchFound = true;
- appendToResult(i, tIndex, writer);
- }
- } while (true);
-
+ boolean matchFound = false;
+ if(tableSize != 0){
+ int entry = tpcProbe.partition(accessorProbe, i, tableSize);
+ int offset = 0;
+ do {
+ table.getTuplePointer(entry, offset++, storedTuplePointer);
+ if (storedTuplePointer.frameIndex < 0)
+ break;
+ int bIndex = storedTuplePointer.frameIndex;
+ int tIndex = storedTuplePointer.tupleIndex;
+ accessorBuild.reset(buffers.get(bIndex));
+ int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
+ if (c == 0) {
+ matchFound = true;
+ appendToResult(i, tIndex, writer);
+ }
+ } while (true);
+ }
+
if (!matchFound && isLeftOuter) {
-
if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
flushFrame(outBuffer, writer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index cf39416..7067080 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -31,7 +31,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -372,9 +371,7 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if(!state.hybridHJ.isTableEmpty()){
- state.hybridHJ.probe(buffer, writer);
- }
+ state.hybridHJ.probe(buffer, writer);
}
@Override
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 354aa1e..5027dc7 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -82,6 +82,16 @@
}
@Override
+ public int getMaxTupleSize(int pageSize) {
+ return (pageSize - getPageHeaderSize()) / 2;
+ }
+
+ @Override
+ public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+ return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
+ }
+
+ @Override
public void setPage(ICachedPage page) {
this.page = page;
this.buf = page.getBuffer();
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
index 90b167f..8753d6b 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
@@ -54,6 +54,11 @@
}
@Override
+ public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+ return tupleWriter.bytesRequired(tuple) + childPtrSize + slotManager.getSlotSize();
+ }
+
+ @Override
public void initBuffer(byte level) {
super.initBuffer(level);
buf.putInt(rightLeafOff, -1);
@@ -185,7 +190,7 @@
ITreeIndexFrame targetFrame = null;
int totalSize = 0;
- int halfPageSize = buf.capacity() / 2 - getPageHeaderSize();
+ int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
int i;
for (i = 0; i < tupleCount; ++i) {
frameTuple.resetByTupleIndex(this, i);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
index 04b3077..bfb1eca 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
@@ -46,6 +46,11 @@
}
@Override
+ public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+ return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
+ }
+
+ @Override
public void initBuffer(byte level) {
super.initBuffer(level);
buf.putInt(nextLeafOff, -1);
@@ -146,7 +151,7 @@
int tuplesToLeft;
ITreeIndexFrame targetFrame = null;
int totalSize = 0;
- int halfPageSize = buf.capacity() / 2 - getPageHeaderSize();
+ int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
int i;
for (i = 0; i < tupleCount; ++i) {
frameTuple.resetByTupleIndex(this, i);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 86bc32a..8846873 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -47,6 +47,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
@@ -74,6 +75,7 @@
private final AtomicInteger smoCounter;
private final ReadWriteLock treeLatch;
+ private final int maxTupleSize;
public BTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IFreePageManager freePageManager,
ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
@@ -82,6 +84,10 @@
fieldCount, file);
this.treeLatch = new ReentrantReadWriteLock(true);
this.smoCounter = new AtomicInteger();
+ ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+ ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+ maxTupleSize = Math.min(leafFrame.getMaxTupleSize(bufferCache.getPageSize()),
+ interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
}
private void diskOrderScan(ITreeIndexCursor icursor, BTreeOpContext ctx) throws HyracksDataException {
@@ -304,11 +310,23 @@
}
private void insert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+ ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+ if (tupleSize > maxTupleSize) {
+ throw new TreeIndexException("Space required for record (" + tupleSize
+ + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+ }
ctx.modificationCallback.before(tuple);
insertUpdateOrDelete(tuple, ctx);
}
private void upsert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+ ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+ if (tupleSize > maxTupleSize) {
+ throw new TreeIndexException("Space required for record (" + tupleSize
+ + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+ }
ctx.modificationCallback.before(tuple);
insertUpdateOrDelete(tuple, ctx);
}
@@ -320,6 +338,12 @@
if (fieldCount == ctx.cmp.getKeyFieldCount()) {
throw new BTreeNotUpdateableException("Cannot perform updates when the entire tuple forms the key.");
}
+ int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+ ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+ if (tupleSize > maxTupleSize) {
+ throw new TreeIndexException("Space required for record (" + tupleSize
+ + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+ }
ctx.modificationCallback.before(tuple);
insertUpdateOrDelete(tuple, ctx);
}
@@ -933,6 +957,13 @@
@Override
public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
try {
+ int tupleSize = Math.max(leafFrame.getBytesRequriedToWriteTuple(tuple),
+ interiorFrame.getBytesRequriedToWriteTuple(tuple));
+ if (tupleSize > maxTupleSize) {
+ throw new TreeIndexException("Space required for record (" + tupleSize
+ + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+ }
+
NodeFrontier leafFrontier = nodeFrontiers.get(0);
int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
index 612af25..4a14505 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
@@ -59,6 +59,10 @@
public ByteBuffer getBuffer();
+ public int getMaxTupleSize(int pageSize);
+
+ public int getBytesRequriedToWriteTuple(ITupleReference tuple);
+
// for debugging
public String printHeader();
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
index e46efff..22c6637 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
+import java.io.File;
import java.io.IOException;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -23,6 +24,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
import edu.uci.ics.hyracks.storage.common.file.LocalResource;
@@ -37,6 +39,7 @@
protected final ResourceIdFactory resourceIdFactory;
protected final FileReference file;
protected final int partition;
+ protected final int ioDeviceId;
protected IIndex index;
@@ -47,7 +50,9 @@
this.localResourceRepository = opDesc.getStorageManager().getLocalResourceRepository(ctx);
this.resourceIdFactory = opDesc.getStorageManager().getResourceIdFactory(ctx);
this.partition = partition;
- this.file = opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile();
+ this.ioDeviceId = opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId();
+ this.file = new FileReference(new File(IndexFileNameUtil.prepareFileName(opDesc.getFileSplitProvider()
+ .getFileSplits()[partition].getLocalFile().getFile().getPath(), ioDeviceId)));
}
protected abstract IIndex createIndexInstance() throws HyracksDataException;
@@ -70,7 +75,7 @@
// any physical artifact that the LocalResourceRepository is managing (e.g. a file containing the resource ID).
// Once the index has been created, a new resource ID can be generated.
if (resourceID != -1) {
- localResourceRepository.deleteResourceByName(file.getFile().getPath());
+ localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
}
index.create();
try {
@@ -78,8 +83,9 @@
resourceID = resourceIdFactory.createId();
ILocalResourceFactory localResourceFactory = opDesc.getLocalResourceFactoryProvider()
.getLocalResourceFactory();
- localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
- .getPath(), partition));
+ localResourceRepository.insert(
+ localResourceFactory.createLocalResource(resourceID, file.getFile().getPath(), partition),
+ ioDeviceId);
} catch (IOException e) {
throw new HyracksDataException(e);
}
@@ -121,7 +127,7 @@
}
if (resourceID != -1) {
- localResourceRepository.deleteResourceByName(file.getFile().getPath());
+ localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
}
index.destroy();
}
@@ -143,4 +149,4 @@
public IHyracksTaskContext getTaskContext() {
return ctx;
}
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
index 31ce573..2e97b21 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
@@ -61,6 +61,11 @@
}
@Override
+ public int getMaxTupleSize(int pageSize) {
+ return (pageSize - getPageHeaderSize()) / 2;
+ }
+
+ @Override
public boolean isLeaf() {
return buf.get(levelOff) == 0;
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexFileNameUtil.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexFileNameUtil.java
new file mode 100644
index 0000000..fbab3cf
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexFileNameUtil.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.util;
+
+import java.io.File;
+
+public class IndexFileNameUtil {
+
+ public static String prepareFileName(String path, int ioDeviceId) {
+ return path + File.separator + "device_id_" + ioDeviceId;
+ }
+}
\ No newline at end of file
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 a85ada6..3ffd35b 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
@@ -53,6 +53,7 @@
.getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc
.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
.getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
- ioScheduler, ioOpCallbackProvider, partition);
+ ioScheduler, ioOpCallbackProvider, opDesc.getFileSplitProvider().getFileSplits()[partition]
+ .getIODeviceId());
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
index bbcee06..0080c42 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -41,8 +40,8 @@
private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
public LSMBTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> btreeFactory, int startIODeviceIndex) {
- super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+ TreeIndexFactory<? extends ITreeIndex> btreeFactory, int ioDeviceId) {
+ super(ioManager, fileMapProvider, file, null, ioDeviceId);
this.btreeFactory = btreeFactory;
}
@@ -80,17 +79,16 @@
ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
- // Gather files from all IODeviceHandles.
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- // List of valid BTree files.
- cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
- HashSet<String> btreeFilesSet = new HashSet<String>();
- for (ComparableFileName cmpFileName : allBTreeFiles) {
- int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
- btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
- }
- validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+ // Gather files from the IODeviceHandle.
+
+ // List of valid BTree files.
+ cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+ HashSet<String> btreeFilesSet = new HashSet<String>();
+ for (ComparableFileName cmpFileName : allBTreeFiles) {
+ int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+ btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
}
+ validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
// Sanity check.
if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/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 c0905ac..d9f2838 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
@@ -61,7 +61,7 @@
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
- ILSMIOOperationCallbackProvider ioOpCallbackProvider, int startIODeviceIndex) {
+ ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) {
LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
cmpFactories.length, false);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -86,7 +86,7 @@
bloomFilterKeyFields);
ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
- diskBTreeFactory, startIODeviceIndex);
+ diskBTreeFactory, ioDeviceId);
LSMBTree lsmTree = new LSMBTree(virtualBufferCache, interiorFrameFactory, insertLeafFrameFactory,
deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index a808143..99b62d8 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -44,10 +44,8 @@
protected static final String SPLIT_STRING = "_";
protected static final String BLOOM_FILTER_STRING = "f";
- // Use all IODevices registered in ioManager in a round-robin fashion to choose
- // where to flush and merge
- protected final IIOManager ioManager;
protected final IFileMapProvider fileMapProvider;
+ protected final IODeviceHandle dev;
// baseDir should reflect dataset name and partition name.
protected String baseDir;
@@ -57,19 +55,15 @@
protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
- // The current index for the round-robin file assignment
- private int ioDeviceIndex = 0;
-
public AbstractLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> treeFactory, int startIODeviceIndex) {
+ TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
this.baseDir = file.getFile().getPath();
if (!baseDir.endsWith(System.getProperty("file.separator"))) {
baseDir += System.getProperty("file.separator");
}
this.fileMapProvider = fileMapProvider;
- this.ioManager = ioManager;
this.treeFactory = treeFactory;
- ioDeviceIndex = startIODeviceIndex % ioManager.getIODevices().size();
+ this.dev = ioManager.getIODevices().get(ioDeviceId);
}
private static FilenameFilter fileNameFilter = new FilenameFilter() {
@@ -135,18 +129,14 @@
@Override
public void createDirs() {
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File f = new File(dev.getPath(), baseDir);
- f.mkdirs();
- }
+ File f = new File(dev.getPath(), baseDir);
+ f.mkdirs();
}
@Override
public void deleteDirs() {
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File f = new File(dev.getPath(), baseDir);
- delete(f);
- }
+ File f = new File(dev.getPath(), baseDir);
+ delete(f);
}
private void delete(File f) {
@@ -165,9 +155,6 @@
};
protected FileReference createFlushFile(String relFlushFileName) {
- // Assigns new files to I/O devices in round-robin fashion.
- IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceIndex);
- ioDeviceIndex = (ioDeviceIndex + 1) % ioManager.getIODevices().size();
return dev.createFileReference(relFlushFileName);
}
@@ -198,14 +185,12 @@
List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
ArrayList<ComparableFileName> allFiles = new ArrayList<ComparableFileName>();
- // Gather files from all IODeviceHandles and delete invalid files
+ // Gather files from the IODeviceHandle and delete invalid files
// There are two types of invalid files:
// (1) The isValid flag is not set
// (2) The file's interval is contained by some other file
// Here, we only filter out (1).
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
- }
+ cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
if (allFiles.isEmpty()) {
return validFiles;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index be29114..cfaae5d 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
@@ -56,11 +56,12 @@
IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache,
- diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
- invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
- invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
- diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
- mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, partition);
+ diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc
+ .getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(), invIndexOpDesc
+ .getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(), diskBufferCache, ctx
+ .getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+ opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+ opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
return invIndex;
} catch (IndexException e) {
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index ab7290e..8b3de17 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
@@ -56,11 +56,12 @@
IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
PartitionedLSMInvertedIndex invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(
- virtualBufferCache, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
- invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
- invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
- diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
- mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, partition);
+ virtualBufferCache, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc
+ .getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(), invIndexOpDesc
+ .getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(), diskBufferCache, ctx
+ .getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+ opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+ opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
return invIndex;
} catch (IndexException e) {
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
index ccba624..aa1dbeb 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
@@ -63,8 +62,8 @@
};
public LSMInvertedIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- BTreeFactory btreeFactory, int startIODeviceIndex) {
- super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+ BTreeFactory btreeFactory, int ioDeviceId) {
+ super(ioManager, fileMapProvider, file, null, ioDeviceId);
this.btreeFactory = btreeFactory;
}
@@ -100,20 +99,19 @@
ArrayList<ComparableFileName> allDeletedKeysBTreeFiles = new ArrayList<ComparableFileName>();
ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
- // Gather files from all IODeviceHandles.
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
- HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
- for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
- int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
- deletedKeysBTreeFilesSet.add(cmpFileName.fileName.substring(0, index));
- }
-
- // TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
- validateFiles(dev, deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
- validateFiles(dev, deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
- validateFiles(dev, deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+ // Gather files from the IODeviceHandle.
+ cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
+ HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
+ for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
+ int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+ deletedKeysBTreeFilesSet.add(cmpFileName.fileName.substring(0, index));
}
+
+ // TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
+ validateFiles(dev, deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
+ validateFiles(dev, deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
+ validateFiles(dev, deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+
// Sanity check.
if (allDictBTreeFiles.size() != allInvListsFiles.size()
|| allDictBTreeFiles.size() != allDeletedKeysBTreeFiles.size()
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 6fd9dbe..58a87b1 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
@@ -135,8 +135,8 @@
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- int startIODeviceIndex) throws IndexException {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
+ throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -150,7 +150,7 @@
FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
- onDiskDirFileRef, deletedKeysBTreeFactory, startIODeviceIndex);
+ onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
invListTypeTraits);
@@ -185,8 +185,8 @@
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- int startIODeviceIndex) throws IndexException {
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
+ throws IndexException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
invListCmpFactories, diskBufferCache);
@@ -200,7 +200,7 @@
FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
- onDiskDirFileRef, deletedKeysBTreeFactory, startIODeviceIndex);
+ onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
invListTypeTraits);
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index 4c112f4..23c1ed4 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
@@ -71,7 +71,8 @@
return LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider, linearizeCmpFactory, startIODeviceIndex);
+ ioOpCallbackProvider, linearizeCmpFactory,
+ opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
} catch (TreeIndexException e) {
throw new HyracksDataException(e);
}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
index 851235e..fd36920 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -56,8 +55,8 @@
public LSMRTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory,
- int startIODeviceIndex) {
- super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+ int ioDeviceId) {
+ super(ioManager, fileMapProvider, file, null, ioDeviceId);
this.rtreeFactory = rtreeFactory;
this.btreeFactory = btreeFactory;
}
@@ -93,17 +92,16 @@
ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
- // Gather files from all IODeviceHandles.
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
- HashSet<String> btreeFilesSet = new HashSet<String>();
- for (ComparableFileName cmpFileName : allBTreeFiles) {
- int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
- btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
- }
- validateFiles(dev, btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
- validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+ // Gather files from the IODeviceHandle.
+ cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+ HashSet<String> btreeFilesSet = new HashSet<String>();
+ for (ComparableFileName cmpFileName : allBTreeFiles) {
+ int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+ btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
}
+ validateFiles(dev, btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
+ validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+
// Sanity check.
if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
throw new HyracksDataException(
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
index 6ddf766..872d152 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
@@ -25,7 +25,7 @@
public class LSMRTreeWithAntiMatterTuplesFileManager extends AbstractLSMIndexFileManager {
public LSMRTreeWithAntiMatterTuplesFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider,
- FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int startIODeviceIndex) {
- super(ioManager, fileMapProvider, file, rtreeFactory, startIODeviceIndex);
+ FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int ioDeviceId) {
+ super(ioManager, fileMapProvider, file, rtreeFactory, ioDeviceId);
}
}
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 f326e3b..6ecd44f 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
@@ -80,7 +80,7 @@
RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
- int startIODeviceIndex) throws TreeIndexException {
+ int ioDeviceId) throws TreeIndexException {
LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
@@ -114,7 +114,7 @@
bloomFilterKeyFields);
ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(ioManager, diskFileMapProvider, file,
- diskRTreeFactory, diskBTreeFactory, startIODeviceIndex);
+ diskRTreeFactory, diskBTreeFactory, ioDeviceId);
LSMRTree lsmTree = new LSMRTree(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
@@ -144,8 +144,7 @@
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
- ILinearizeComparatorFactory linearizerCmpFactory, int startIODeviceIndex) throws TreeIndexException {
-
+ ILinearizeComparatorFactory linearizerCmpFactory, int ioDeviceId) throws TreeIndexException {
LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
@@ -181,7 +180,7 @@
btreeCmpFactories[btreeCmpFactories.length - 1] };
ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(ioManager,
- diskFileMapProvider, file, diskRTreeFactory, startIODeviceIndex);
+ diskFileMapProvider, file, diskRTreeFactory, ioDeviceId);
LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(virtualBufferCache,
rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
index 5ab9632..f197853 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
@@ -48,6 +48,11 @@
}
@Override
+ public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+ return tupleWriter.bytesRequired(tuple) + childPtrSize + slotManager.getSlotSize();
+ }
+
+ @Override
public int findBestChild(ITupleReference tuple, MultiComparator cmp) {
int bestChild = rtreePolicy.findBestChildPosition(this, tuple, frameTuple, cmp);
frameTuple.resetByTupleIndex(this, bestChild);
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
index d52ef16..4057778 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
@@ -30,6 +30,11 @@
}
@Override
+ public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+ return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
+ }
+
+ @Override
public ITreeIndexTupleReference createTupleReference() {
return tupleWriter.createTupleReference();
}
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index c12dc50..48fb6f1 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -61,12 +61,18 @@
// Global node sequence number used for the concurrency control protocol
private final AtomicLong globalNsn;
+ private final int maxTupleSize;
+
public RTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IFreePageManager freePageManager,
ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
super(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
fieldCount, file);
globalNsn = new AtomicLong();
+ ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+ ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+ maxTupleSize = Math.min(leafFrame.getMaxTupleSize(bufferCache.getPageSize()),
+ interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
}
private long incrementGlobalNsn() {
@@ -147,6 +153,12 @@
private void insert(ITupleReference tuple, IIndexOperationContext ictx) throws HyracksDataException,
TreeIndexException {
RTreeOpContext ctx = (RTreeOpContext) ictx;
+ int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+ ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+ if (tupleSize > maxTupleSize) {
+ throw new TreeIndexException("Record size (" + tupleSize + ") larger than maximum acceptable record size ("
+ + maxTupleSize + ")");
+ }
ctx.reset();
ctx.setTuple(tuple);
ctx.splitKey.reset();
@@ -614,7 +626,7 @@
ctx.splitKey.reset();
ctx.splitKey.getLeftTuple().setFieldCount(cmpFactories.length);
- // We delete the first matching tuple (including the payload data.
+ // We delete the first matching tuple (including the payload data).
// We don't update the MBRs of the parents after deleting the record.
int tupleIndex = findTupleToDelete(ctx);
@@ -870,8 +882,15 @@
}
@Override
- public void add(ITupleReference tuple) throws HyracksDataException {
+ public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
try {
+ int tupleSize = Math.max(leafFrame.getBytesRequriedToWriteTuple(tuple),
+ interiorFrame.getBytesRequriedToWriteTuple(tuple));
+ if (tupleSize > maxTupleSize) {
+ throw new TreeIndexException("Space required for record (" + tupleSize
+ + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+ }
+
NodeFrontier leafFrontier = nodeFrontiers.get(0);
int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
index ab9ec41..36aa088 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
@@ -19,16 +19,16 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface ILocalResourceRepository {
-
+
public LocalResource getResourceById(long id) throws HyracksDataException;
public LocalResource getResourceByName(String name) throws HyracksDataException;
- public void insert(LocalResource resource) throws HyracksDataException;
+ public void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException;
- public void deleteResourceById(long id) throws HyracksDataException;
+ public void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException;
- public void deleteResourceByName(String name) throws HyracksDataException;
+ public void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException;
public List<LocalResource> getAllResources() throws HyracksDataException;
}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
index 55bd807..f853a36 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
@@ -37,7 +37,7 @@
}
@Override
- public synchronized void insert(LocalResource resource) throws HyracksDataException {
+ public synchronized void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException {
long id = resource.getResourceId();
if (id2ResourceMap.containsKey(id)) {
@@ -48,7 +48,7 @@
}
@Override
- public synchronized void deleteResourceById(long id) throws HyracksDataException {
+ public synchronized void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException {
LocalResource resource = id2ResourceMap.get(id);
if (resource == null) {
throw new HyracksDataException("Resource doesn't exist");
@@ -58,7 +58,7 @@
}
@Override
- public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+ public synchronized void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException {
LocalResource resource = name2ResourceMap.get(name);
if (resource == null) {
throw new HyracksDataException("Resource doesn't exist");
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index f93e9b6..fdb4341 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -225,7 +225,10 @@
int p1y = rnd.nextInt();
int p2x = rnd.nextInt();
int p2y = rnd.nextInt();
- String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ String data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
@@ -261,7 +264,10 @@
p1y = rnd.nextInt();
p2x = rnd.nextInt();
p2y = rnd.nextInt();
- data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
@@ -270,7 +276,10 @@
p1y = rnd.nextInt();
p2x = rnd.nextInt();
p2y = rnd.nextInt();
- data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
@@ -340,7 +349,10 @@
int p1y = rnd.nextInt();
int p2x = rnd.nextInt();
int p2y = rnd.nextInt();
- String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ String data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
@@ -349,7 +361,10 @@
p1y = rnd.nextInt();
p2x = rnd.nextInt();
p2y = rnd.nextInt();
- data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
@@ -376,7 +391,10 @@
p1y = rnd.nextInt();
p2x = rnd.nextInt();
p2y = rnd.nextInt();
- data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
@@ -385,7 +403,10 @@
p1y = rnd.nextInt();
p2x = rnd.nextInt();
p2y = rnd.nextInt();
- data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+ data = "";
+ for (int i = 0; i < 210; i++) {
+ data += "X";
+ }
TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
Math.max(p1y, p2y), data);
indexAccessor.insert(tuple);
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 280a8b3..dc8d6a1 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
@@ -55,7 +55,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 9edcd4c..880ed5a 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
@@ -55,7 +55,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 727e50b..72add94 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
@@ -38,7 +38,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 349acd4..7dc4492 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
@@ -55,7 +55,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 c389258..e5cfef5 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
@@ -46,7 +46,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, fieldSerdes.length, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
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 5ae3d62..7fcf416 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
@@ -54,7 +54,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 763a81e..1c4542f 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(),
NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 38053e4..e645f85 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
@@ -56,7 +56,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 3a305a2..4f4ac87 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
@@ -41,7 +41,7 @@
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 cc20fc4..73bce86 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
@@ -55,7 +55,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 71b7bde..103059f 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
@@ -54,7 +54,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 1b9e8a9..1cd427f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -53,6 +53,7 @@
protected IHyracksTaskContext ctx;
protected IOManager ioManager;
+ protected int ioDeviceId;
protected IBufferCache bufferCache;
protected int lsmtreeFileId;
@@ -84,6 +85,7 @@
TestStorageManagerComponentHolder.init(this.onDiskPageSize, this.onDiskNumPages, MAX_OPEN_FILES);
bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
@@ -91,7 +93,8 @@
this.ioScheduler = SynchronousScheduler.INSTANCE;
lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache, fmp, typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
- ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE);
+ ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE,
+ ioDeviceId);
}
@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 9978689..0cbef61 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
@@ -66,7 +66,7 @@
FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
ISerializerDeserializer[] fieldSerdes, int numKeyFields, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -76,7 +76,7 @@
}
LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
- mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+ mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, ioDeviceId);
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 584a39f..08a8ebd 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
@@ -62,6 +62,7 @@
protected final double bloomFilterFalsePositiveRate;
protected IOManager ioManager;
+ protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
protected IVirtualBufferCache virtualBufferCache;
@@ -114,6 +115,7 @@
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
}
@@ -165,6 +167,10 @@
return ioManager;
}
+ public int getIODeviceId() {
+ return ioDeviceId;
+ }
+
public IBufferCache getDiskBufferCache() {
return diskBufferCache;
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
index 69e23bc..d41ddd4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
@@ -32,8 +32,8 @@
public class DummyLSMIndexFileManager extends AbstractLSMIndexFileManager {
public DummyLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
- TreeIndexFactory<? extends ITreeIndex> treeFactory) {
- super(ioManager, fileMapProvider, file, treeFactory, 0);
+ TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
+ super(ioManager, fileMapProvider, file, treeFactory, ioDeviceId);
}
protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
index 161f4ce..0c9e658 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
@@ -48,6 +48,7 @@
private static final int DEFAULT_PAGE_SIZE = 256;
private static final int DEFAULT_NUM_PAGES = 100;
private static final int DEFAULT_MAX_OPEN_FILES = 10;
+ private static final int DEFAULT_IO_DEVICE_ID = 0;
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
protected final static String sep = System.getProperty("file.separator");
protected IOManager ioManager;
@@ -74,7 +75,7 @@
public void sortOrderTest(boolean testFlushFileName) throws InterruptedException, HyracksDataException {
ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
- new DummyTreeFactory());
+ new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
LinkedList<String> fileNames = new LinkedList<String>();
int numFileNames = 100;
@@ -115,7 +116,7 @@
public void cleanInvalidFilesTest(IOManager ioManager) throws InterruptedException, IOException, IndexException {
ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
- new DummyTreeFactory());
+ new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
fileManager.createDirs();
List<FileReference> flushFiles = new ArrayList<FileReference>();
@@ -187,20 +188,18 @@
.getFile().getName());
}
- // Make sure invalid files were removed from all IODevices.
+ // Make sure invalid files were removed from the IODevices.
ArrayList<String> remainingFiles = new ArrayList<String>();
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File dir = new File(dev.getPath(), baseDir);
- FilenameFilter filter = new FilenameFilter() {
- public boolean accept(File dir, String name) {
- return !name.startsWith(".");
- }
- };
- String[] files = dir.list(filter);
- for (String file : files) {
- File f = new File(file);
- remainingFiles.add(f.getName());
+ File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+ FilenameFilter filter = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ return !name.startsWith(".");
}
+ };
+ String[] files = dir.list(filter);
+ for (String file : files) {
+ File f = new File(file);
+ remainingFiles.add(f.getName());
}
Collections.sort(remainingFiles, fileManager.getFileNameComparator());
@@ -218,34 +217,19 @@
cleanDirs(singleDeviceIOManager);
}
- @Test
- public void twoIODevicesTest() throws InterruptedException, IOException, IndexException {
- IOManager twoDevicesIOManager = createIOManager(2);
- cleanInvalidFilesTest(twoDevicesIOManager);
- cleanDirs(twoDevicesIOManager);
- }
-
- @Test
- public void fourIODevicesTest() throws InterruptedException, IOException, IndexException {
- IOManager fourDevicesIOManager = createIOManager(4);
- cleanInvalidFilesTest(fourDevicesIOManager);
- cleanDirs(fourDevicesIOManager);
- }
-
private void cleanDirs(IOManager ioManager) {
- for (IODeviceHandle dev : ioManager.getIODevices()) {
- File dir = new File(dev.getPath(), baseDir);
- FilenameFilter filter = new FilenameFilter() {
- public boolean accept(File dir, String name) {
- return !name.startsWith(".");
- }
- };
- String[] files = dir.list(filter);
- for (String file : files) {
- File f = new File(file);
- f.delete();
+ File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+ FilenameFilter filter = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ return !name.startsWith(".");
}
+ };
+ String[] files = dir.list(filter);
+ for (String file : files) {
+ File f = new File(file);
+ f.delete();
}
+
}
private IOManager createIOManager(int numDevices) throws HyracksException {
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 3ab197c..d5f735c 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
@@ -59,6 +59,7 @@
protected final double bloomFilterFalsePositiveRate;
protected IOManager ioManager;
+ protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
protected IVirtualBufferCache virtualBufferCache;
@@ -115,6 +116,7 @@
virtualBufferCache.open();
virtualFreePageManager = new VirtualFreePageManager(memNumPages);
ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
invIndexFileRef = ioManager.getIODevices().get(0).createFileReference(onDiskDir + invIndexFileName);
}
@@ -172,6 +174,10 @@
return ioManager;
}
+ public int getIODeviceId() {
+ return ioDeviceId;
+ }
+
public IBufferCache getDiskBufferCache() {
return diskBufferCache;
}
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 be02283..cf3dd6d 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
@@ -150,7 +150,7 @@
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
break;
}
case PARTITIONED_LSM: {
@@ -159,7 +159,7 @@
tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
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 938b90c..0cb6dbe 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
@@ -59,7 +59,7 @@
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 adcb000..62acff6 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
@@ -59,7 +59,7 @@
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 2fcb96b..135ec77 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
@@ -43,7 +43,7 @@
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
@Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 07ff03d..a90edd5 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
@@ -59,7 +59,7 @@
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 ffb5c78..40dd41f 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
@@ -57,7 +57,7 @@
fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
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 cadebb4..87d50d0 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
@@ -58,7 +58,7 @@
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
index 50578cc..e2c6b68 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
@@ -59,7 +59,7 @@
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 d4a1bc2..d994b7d 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
@@ -58,8 +58,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
-
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 e278133..0d209cdd 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
@@ -58,8 +58,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
-
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 e64046d..976463b 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
@@ -42,7 +42,7 @@
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
@Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index 2e984a0..36afe51 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
@@ -58,8 +58,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
-
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 be9e765..70d53b9 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
@@ -56,7 +56,7 @@
harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
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 a4dc608..e9e08a0 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
@@ -57,7 +57,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
index 2e4ffe2..8b3a6cd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
@@ -58,8 +58,7 @@
harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
- harness.getIOOperationCallbackProvider());
-
+ harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
}
@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 ac8a3c0..4af953b 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
@@ -63,7 +63,7 @@
harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
harness.getOperationTrackerFactory(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
@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/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 9a39f9e..7380118 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
@@ -62,7 +62,7 @@
rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
harness.getIOOperationCallbackProvider(),
- LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+ LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
}
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 f81f28c..cda3d6e 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
@@ -71,7 +71,7 @@
ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
int numKeyFields, RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
@@ -81,7 +81,8 @@
LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+ ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
+ ioDeviceId);
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 1571fff..aea8bbf 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
@@ -59,6 +59,7 @@
protected final double bloomFilterFalsePositiveRate;
protected IOManager ioManager;
+ protected int ioDeviceId;
protected IBufferCache diskBufferCache;
protected IFileMapProvider diskFileMapProvider;
protected IVirtualBufferCache virtualBufferCache;
@@ -111,6 +112,7 @@
diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
ioManager = TestStorageManagerComponentHolder.getIOManager();
+ ioDeviceId = 0;
rnd.setSeed(RANDOM_SEED);
}
@@ -162,6 +164,10 @@
return ioManager;
}
+ public int getIODeviceId() {
+ return ioDeviceId;
+ }
+
public IBufferCache getDiskBufferCache() {
return diskBufferCache;
}
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 95ae1d0..b688b09 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
@@ -71,7 +71,7 @@
IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+ ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
@@ -81,7 +81,8 @@
LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache,
ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories,
btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
- ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+ ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
+ ioDeviceId);
LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
lsmTree);
return testCtx;
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 b9d1d74..9d23ca8 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
@@ -230,10 +230,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -453,10 +454,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
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 bc5ab14..250d245 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
@@ -213,10 +213,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -414,9 +415,10 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
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 a47101d..f796dd7 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
@@ -205,10 +205,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -391,10 +392,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
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 96cfaba..cb6c215 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
@@ -219,10 +219,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */
@@ -424,10 +425,11 @@
/**
* add the delete operator to delete vertexes
*/
+ int[] fieldPermutationDelete = new int[] { 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
- null, NoOpOperationCallbackFactory.INSTANCE);
+ comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+ new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
ClusterConfig.setLocationConstraint(spec, deleteOp);
/** construct empty sink operator */